merge hyracks_dev_next r847:977
git-svn-id: https://hyracks.googlecode.com/svn/branches/aggregators_dev_next@978 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-api/pom.xml b/hyracks-api/pom.xml
index f73c27a..c18995d 100644
--- a/hyracks-api/pom.xml
+++ b/hyracks-api/pom.xml
@@ -1,9 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-api</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks</artifactId>
@@ -45,5 +42,15 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-ipc</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ <version>3.1</version>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index 6e1eedc..866d307 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.hyracks.api.client;
-import java.rmi.Remote;
import java.util.EnumSet;
import java.util.Map;
@@ -22,7 +21,7 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
-public interface IHyracksClientInterface extends Remote {
+public interface IHyracksClientInterface {
public ClusterControllerInfo getClusterControllerInfo() throws Exception;
public void createApplication(String appName) throws Exception;
@@ -35,7 +34,7 @@
public JobStatus getJobStatus(JobId jobId) throws Exception;
- public void start(JobId jobId) throws Exception;
+ public void startJob(JobId jobId) throws Exception;
public void waitForCompletion(JobId jobId) throws Exception;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index c6ca51c..75cc245 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -52,6 +52,16 @@
}
public String toString() {
- return "ANID:[" + odId + "]:" + id;
+ return "ANID:" + odId + ":" + id;
+ }
+
+ public static ActivityId parse(String str) {
+ if (str.startsWith("ANID:")) {
+ str = str.substring(5);
+ int idIdx = str.lastIndexOf(':');
+ return new ActivityId(OperatorDescriptorId.parse(str.substring(0, idIdx)), Integer.parseInt(str
+ .substring(idIdx + 1)));
+ }
+ throw new IllegalArgumentException("Unable to parse: " + str);
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
index b858736..8794e09 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
@@ -49,4 +49,12 @@
public String toString() {
return "ODID:" + id;
}
+
+ public static OperatorDescriptorId parse(String str) {
+ if (str.startsWith("ODID:")) {
+ str = str.substring(5);
+ return new OperatorDescriptorId(Integer.parseInt(str));
+ }
+ throw new IllegalArgumentException("Unable to parse: " + str);
+ }
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
index d4e6972..0fb44c1 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
@@ -52,6 +52,15 @@
@Override
public String toString() {
- return "TAID:[" + taskId + "]:" + attempt;
+ return "TAID:" + taskId + ":" + attempt;
+ }
+
+ public static TaskAttemptId parse(String str) {
+ if (str.startsWith("TAID:")) {
+ str = str.substring(5);
+ int idIdx = str.lastIndexOf(':');
+ return new TaskAttemptId(TaskId.parse(str.substring(0, idIdx)), Integer.parseInt(str.substring(idIdx + 1)));
+ }
+ throw new IllegalArgumentException("Unable to parse: " + str);
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
index ee63355..7e0b22d 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
@@ -52,6 +52,15 @@
@Override
public String toString() {
- return "TID:[" + activityId + "]:" + partition;
+ return "TID:" + activityId + ":" + partition;
+ }
+
+ public static TaskId parse(String str) {
+ if (str.startsWith("TID:")) {
+ str = str.substring(4);
+ int idIdx = str.lastIndexOf(':');
+ return new TaskId(ActivityId.parse(str.substring(0, idIdx)), Integer.parseInt(str.substring(idIdx + 1)));
+ }
+ throw new IllegalArgumentException("Unable to parse: " + str);
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
index 4922723..b04fe86 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
@@ -21,25 +21,29 @@
private static final long serialVersionUID = 1L;
private final ISerializerDeserializer[] fields;
- private final ITypeTrait[] typeTraits;
-
+ private final ITypeTraits[] typeTraits;
+
// leaving this constructor for backwards-compatibility
public RecordDescriptor(ISerializerDeserializer[] fields) {
this.fields = fields;
this.typeTraits = null;
}
-
+
// temporarily adding constructor to include type traits
- public RecordDescriptor(ISerializerDeserializer[] fields, ITypeTrait[] typeTraits) {
- this.fields = fields;
+ public RecordDescriptor(ISerializerDeserializer[] fields, ITypeTraits[] typeTraits) {
+ this.fields = fields;
this.typeTraits = typeTraits;
}
+ public int getFieldCount() {
+ return fields.length;
+ }
+
public ISerializerDeserializer[] getFields() {
return fields;
}
-
- public ITypeTrait[] getTypeTraits() {
- return typeTraits;
+
+ public ITypeTraits[] getTypeTraits() {
+ return typeTraits;
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
index 930d299..2b6d361 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
@@ -22,6 +22,7 @@
import java.util.Map;
import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
import org.json.JSONArray;
import org.json.JSONException;
import org.json.JSONObject;
@@ -33,7 +34,6 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.util.Pair;
public class JobActivityGraph implements Serializable {
private static final long serialVersionUID = 1L;
@@ -148,8 +148,8 @@
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
.getConnectorOperatorMap().get(cdId);
- OperatorDescriptorId consumerOpId = connEdge.second.first.getOperatorId();
- int consumerInputIdx = connEdge.second.second;
+ OperatorDescriptorId consumerOpId = connEdge.getRight().getLeft().getOperatorId();
+ int consumerInputIdx = connEdge.getRight().getRight();
for (ActivityId anId : operatorActivityMap.get(consumerOpId)) {
List<Integer> anInputs = activityInputMap.get(anId);
@@ -168,8 +168,8 @@
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
.getConnectorOperatorMap().get(cdId);
- OperatorDescriptorId producerOpId = connEdge.first.first.getOperatorId();
- int producerInputIdx = connEdge.first.second;
+ OperatorDescriptorId producerOpId = connEdge.getLeft().getLeft().getOperatorId();
+ int producerInputIdx = connEdge.getLeft().getRight();
for (ActivityId anId : operatorActivityMap.get(producerOpId)) {
List<Integer> anOutputs = activityOutputMap.get(anId);
@@ -209,13 +209,11 @@
public JSONObject toJSON() throws JSONException {
JSONObject jplan = new JSONObject();
- jplan.put("type", "plan");
jplan.put("flags", jobFlags.toString());
JSONArray jans = new JSONArray();
for (IActivity an : activityNodes.values()) {
JSONObject jan = new JSONObject();
- jan.put("type", "activity");
jan.put("id", an.getActivityId().toString());
jan.put("java-class", an.getClass().getName());
jan.put("operator-id", an.getActivityId().getOperatorDescriptorId().toString());
@@ -225,7 +223,6 @@
JSONArray jInputs = new JSONArray();
for (int i = 0; i < inputs.size(); ++i) {
JSONObject jInput = new JSONObject();
- jInput.put("type", "activity-input");
jInput.put("input-port", i);
jInput.put("connector-id", inputs.get(i).getConnectorId().toString());
jInputs.put(jInput);
@@ -238,7 +235,6 @@
JSONArray jOutputs = new JSONArray();
for (int i = 0; i < outputs.size(); ++i) {
JSONObject jOutput = new JSONObject();
- jOutput.put("type", "activity-output");
jOutput.put("output-port", i);
jOutput.put("connector-id", outputs.get(i).getConnectorId().toString());
jOutputs.put(jOutput);
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 23b33f0..9c6b88c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -22,6 +22,7 @@
import java.util.Map;
import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
import org.json.JSONArray;
import org.json.JSONException;
import org.json.JSONObject;
@@ -33,7 +34,6 @@
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.util.Pair;
public class JobSpecification implements Serializable {
private static final long serialVersionUID = 1L;
@@ -94,10 +94,11 @@
IOperatorDescriptor consumerOp, int consumerPort) {
insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
insertIntoIndexedMap(opOutputMap, producerOp.getOperatorId(), producerPort, conn);
- connectorOpMap.put(conn.getConnectorId(),
- new Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>(
- new Pair<IOperatorDescriptor, Integer>(producerOp, producerPort),
- new Pair<IOperatorDescriptor, Integer>(consumerOp, consumerPort)));
+ connectorOpMap.put(
+ conn.getConnectorId(),
+ Pair.<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> of(
+ Pair.<IOperatorDescriptor, Integer> of(producerOp, producerPort),
+ Pair.<IOperatorDescriptor, Integer> of(consumerOp, consumerPort)));
}
public void setProperty(String name, Serializable value) {
@@ -126,19 +127,19 @@
public RecordDescriptor getConnectorRecordDescriptor(IConnectorDescriptor conn) {
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
.getConnectorId());
- return connInfo.first.first.getOutputRecordDescriptors()[connInfo.first.second];
+ return connInfo.getLeft().getLeft().getOutputRecordDescriptors()[connInfo.getLeft().getRight()];
}
public IOperatorDescriptor getConsumer(IConnectorDescriptor conn) {
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
.getConnectorId());
- return connInfo.second.first;
+ return connInfo.getRight().getLeft();
}
public int getConsumerInputIndex(IConnectorDescriptor conn) {
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
.getConnectorId());
- return connInfo.second.second;
+ return connInfo.getRight().getRight();
}
public IConnectorDescriptor getInputConnectorDescriptor(IOperatorDescriptor op, int inputIndex) {
@@ -180,13 +181,13 @@
public IOperatorDescriptor getProducer(IConnectorDescriptor conn) {
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
.getConnectorId());
- return connInfo.first.first;
+ return connInfo.getLeft().getLeft();
}
public int getProducerOutputIndex(IConnectorDescriptor conn) {
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
.getConnectorId());
- return connInfo.first.second;
+ return connInfo.getLeft().getRight();
}
public List<OperatorDescriptorId> getRoots() {
@@ -264,8 +265,6 @@
public JSONObject toJSON() throws JSONException {
JSONObject jjob = new JSONObject();
- jjob.put("type", "job");
-
JSONArray jopArray = new JSONArray();
for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> e : opMap.entrySet()) {
jopArray.put(e.getValue().toJSON());
@@ -277,12 +276,11 @@
JSONObject conn = new JSONObject();
Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connection = connectorOpMap
.get(e.getKey());
- conn.put("type", "connector-info");
if (connection != null) {
- conn.put("in-operator-id", connection.first.first.getOperatorId().toString());
- conn.put("in-operator-port", connection.first.second.intValue());
- conn.put("out-operator-id", connection.second.first.getOperatorId().toString());
- conn.put("out-operator-port", connection.second.second.intValue());
+ conn.put("in-operator-id", connection.getLeft().getLeft().getOperatorId().toString());
+ conn.put("in-operator-port", connection.getLeft().getRight().intValue());
+ conn.put("out-operator-id", connection.getRight().getLeft().getOperatorId().toString());
+ conn.put("out-operator-port", connection.getRight().getRight().intValue());
}
conn.put("connector", e.getValue().toJSON());
jcArray.put(conn);
diff --git a/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java b/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
index e1a1fd2..168280b 100644
--- a/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
+++ b/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
@@ -1,6 +1,6 @@
package edu.uci.ics.hyracks.cli.commands;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.cli.Session;
@@ -12,7 +12,7 @@
public ConnectCommand(String hostPortStr) {
int idx = hostPortStr.indexOf(':');
host = hostPortStr;
- port = 1099;
+ port = 1098;
if (idx != -1) {
host = hostPortStr.substring(0, idx);
port = Integer.valueOf(hostPortStr.substring(idx + 1));
@@ -22,7 +22,7 @@
@Override
public void run(Session session) throws Exception {
System.err.println("Connecting to host: " + host + ", port: " + port);
- IHyracksClientConnection conn = new HyracksRMIConnection(host, port);
+ IHyracksClientConnection conn = new HyracksConnection(host, port);
session.setConnection(conn);
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index bcc3c30..4575adc 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -15,8 +15,7 @@
package edu.uci.ics.hyracks.control.cc;
import java.io.File;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
+import java.net.InetSocketAddress;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Hashtable;
@@ -36,14 +35,15 @@
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
import edu.uci.ics.hyracks.api.context.ICCContext;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.ipc.HyracksClientInterfaceDelegateIPCI;
import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
+import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobStatusConditionVariableWork;
@@ -69,6 +69,8 @@
import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
+import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerDelegateIPCI;
+import edu.uci.ics.hyracks.control.common.ipc.NodeControllerRemoteProxy;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
@@ -76,14 +78,19 @@
import edu.uci.ics.hyracks.control.common.logs.LogFile;
import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.WorkQueue;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
IHyracksClientInterface {
- private static final long serialVersionUID = 1L;
+ private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
private final CCConfig ccConfig;
- private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
+ private IPCSystem clusterIPC;
+
+ private IPCSystem clientIPC;
private final LogFile jobLog;
@@ -105,12 +112,10 @@
private final WorkQueue workQueue;
- private final Executor taskExecutor;
+ private final Executor executor;
private final Timer timer;
- private final CCClientInterface ccci;
-
private final ICCContext ccContext;
private final DeadNodeSweeper sweeper;
@@ -125,7 +130,12 @@
ipAddressNodeNameMap = new HashMap<String, Set<String>>();
applications = new Hashtable<String, CCApplicationContext>();
serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
- taskExecutor = Executors.newCachedThreadPool();
+ executor = Executors.newCachedThreadPool();
+ IIPCI ccIPCI = new ClusterControllerDelegateIPCI(this);
+ clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI, executor);
+ IIPCI ciIPCI = new HyracksClientInterfaceDelegateIPCI(this);
+ clientIPC = new IPCSystem(new InetSocketAddress(ccConfig.clientNetIpAddress, ccConfig.clientNetPort), ciIPCI,
+ executor);
webServer = new WebServer(this);
activeRunMap = new HashMap<JobId, JobRun>();
runMapArchive = new LinkedHashMap<JobId, JobRun>() {
@@ -137,7 +147,6 @@
};
workQueue = new WorkQueue();
this.timer = new Timer(true);
- ccci = new CCClientInterface(this);
ccContext = new ICCContext() {
@Override
public Map<String, Set<String>> getIPAddressNodeMap() {
@@ -151,9 +160,8 @@
@Override
public void start() throws Exception {
LOGGER.log(Level.INFO, "Starting ClusterControllerService: " + this);
- Registry registry = LocateRegistry.createRegistry(ccConfig.port);
- registry.rebind(IHyracksClientInterface.class.getName(), ccci);
- registry.rebind(IClusterController.class.getName(), this);
+ clusterIPC.start();
+ clientIPC.start();
webServer.setPort(ccConfig.httpPort);
webServer.start();
workQueue.start();
@@ -174,6 +182,14 @@
LOGGER.log(Level.INFO, "Stopped ClusterControllerService");
}
+ public ServerContext getServerContext() {
+ return serverCtx;
+ }
+
+ public ICCContext getCCContext() {
+ return ccContext;
+ }
+
public Map<String, CCApplicationContext> getApplicationMap() {
return applications;
}
@@ -195,7 +211,7 @@
}
public Executor getExecutor() {
- return taskExecutor;
+ return executor;
}
public Map<String, NodeControllerState> getNodeMap() {
@@ -221,11 +237,14 @@
@Override
public NodeParameters registerNode(NodeRegistration reg) throws Exception {
- INodeController nodeController = reg.getNodeController();
+ InetSocketAddress ncAddress = reg.getNodeControllerAddress();
String id = reg.getNodeId();
+
+ IIPCHandle ncIPCHandle = clusterIPC.getHandle(reg.getNodeControllerAddress());
+ INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
+
NodeControllerState state = new NodeControllerState(nodeController, reg);
workQueue.scheduleAndSync(new RegisterNodeWork(this, id, state));
- nodeController.notifyRegistration(this);
LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
NodeParameters params = new NodeParameters();
params.setClusterControllerInfo(info);
@@ -235,10 +254,8 @@
}
@Override
- public void unregisterNode(INodeController nodeController) throws Exception {
- String id = nodeController.getId();
- workQueue.scheduleAndSync(new UnregisterNodeWork(this, id));
- LOGGER.log(Level.INFO, "Unregistered INodeController");
+ public void unregisterNode(String nodeId) throws Exception {
+ workQueue.schedule(new UnregisterNodeWork(this, nodeId));
}
@Override
@@ -268,7 +285,7 @@
}
@Override
- public void start(JobId jobId) throws Exception {
+ public void startJob(JobId jobId) throws Exception {
JobStartWork jse = new JobStartWork(this, jobId);
workQueue.schedule(jse);
}
@@ -295,13 +312,9 @@
@Override
public void createApplication(String appName) throws Exception {
- synchronized (applications) {
- if (applications.containsKey(appName)) {
- throw new HyracksException("Duplicate application with name: " + appName + " being created.");
- }
- CCApplicationContext appCtx = new CCApplicationContext(serverCtx, ccContext, appName);
- applications.put(appName, appCtx);
- }
+ FutureValue<Object> fv = new FutureValue<Object>();
+ workQueue.schedule(new ApplicationCreateWork(this, appName, fv));
+ fv.get();
}
@Override
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
index 6b7e143..585c7a1 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
@@ -15,17 +15,22 @@
package edu.uci.ics.hyracks.control.cc.adminconsole.pages;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.wicket.markup.html.basic.Label;
-import org.apache.wicket.markup.html.panel.EmptyPanel;
import org.apache.wicket.request.mapper.parameter.PageParameters;
import org.apache.wicket.util.string.StringValue;
import org.json.JSONArray;
import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.work.GetJobActivityGraphJSONWork;
@@ -35,8 +40,7 @@
public class JobDetailsPage extends AbstractPage {
private static final long serialVersionUID = 1L;
- private static final int HEIGHT = 9;
- private static final int WIDTH = 9;
+ private static final int HEIGHT = 29;
public JobDetailsPage(PageParameters params) throws Exception {
ClusterControllerService ccs = getAdminConsoleApplication().getClusterControllerService();
@@ -47,19 +51,105 @@
GetJobSpecificationJSONWork gjsw = new GetJobSpecificationJSONWork(ccs, jobId);
ccs.getWorkQueue().scheduleAndSync(gjsw);
- add(new Label("job-specification", gjsw.getJSON().toString()));
+ Label jobspec = new Label("job-specification", gjsw.getJSON().toString());
+ jobspec.setEscapeModelStrings(false);
+ add(jobspec);
GetJobActivityGraphJSONWork gjagw = new GetJobActivityGraphJSONWork(ccs, jobId);
ccs.getWorkQueue().scheduleAndSync(gjagw);
- add(new Label("job-activity-graph", gjagw.getJSON().toString()));
+ Label jag = new Label("job-activity-graph", gjagw.getJSON().toString());
+ jag.setEscapeModelStrings(false);
+ add(jag);
+
+ JSONObject jagO = gjagw.getJSON();
+
+ Map<ActivityId, String> activityMap = new HashMap<ActivityId, String>();
+ if (jagO.has("activities")) {
+ JSONArray aArray = jagO.getJSONArray("activities");
+ for (int i = 0; i < aArray.length(); ++i) {
+ JSONObject aO = aArray.getJSONObject(i);
+ ActivityId aid = ActivityId.parse(aO.getString("id"));
+ String className = aO.getString("java-class");
+
+ activityMap.put(aid, className);
+ }
+ }
GetJobRunJSONWork gjrw = new GetJobRunJSONWork(ccs, jobId);
ccs.getWorkQueue().scheduleAndSync(gjrw);
- add(new Label("job-run", gjrw.getJSON().toString()));
+ Label jobrun = new Label("job-run", gjrw.getJSON().toString());
+ jobrun.setEscapeModelStrings(false);
+ add(jobrun);
JSONObject jrO = gjrw.getJSON();
- List<TaskProfile> taskProfiles = new ArrayList<TaskProfile>();
+ List<TaskClusterAttempt[]> tcList = new ArrayList<TaskClusterAttempt[]>();
+ long minTime = Long.MAX_VALUE;
+ long maxTime = Long.MIN_VALUE;
+ if (jrO.has("activity-clusters")) {
+ JSONArray acA = jrO.getJSONArray("activity-clusters");
+ for (int i = 0; i < acA.length(); ++i) {
+ JSONObject acO = acA.getJSONObject(i);
+ if (acO.has("plan")) {
+ JSONObject planO = acO.getJSONObject("plan");
+ if (planO.has("task-clusters")) {
+ JSONArray tcA = planO.getJSONArray("task-clusters");
+ for (int j = 0; j < tcA.length(); ++j) {
+ JSONObject tcO = tcA.getJSONObject(j);
+ String tcId = tcO.getString("task-cluster-id");
+ if (tcO.has("attempts")) {
+ JSONArray tcaA = tcO.getJSONArray("attempts");
+ TaskClusterAttempt[] tcAttempts = new TaskClusterAttempt[tcaA.length()];
+ for (int k = 0; k < tcaA.length(); ++k) {
+ JSONObject tcaO = tcaA.getJSONObject(k);
+ int attempt = tcaO.getInt("attempt");
+ long startTime = tcaO.getLong("start-time");
+ long endTime = tcaO.getLong("end-time");
+
+ tcAttempts[k] = new TaskClusterAttempt(tcId, attempt, startTime, endTime);
+ if (startTime < minTime) {
+ minTime = startTime;
+ }
+ if (endTime > maxTime) {
+ maxTime = endTime;
+ }
+ if (tcaO.has("task-attempts")) {
+ JSONArray taArray = tcaO.getJSONArray("task-attempts");
+ tcAttempts[k].tasks = new TaskAttempt[taArray.length()];
+ for (int l = 0; l < taArray.length(); ++l) {
+ JSONObject taO = taArray.getJSONObject(l);
+ TaskAttemptId taId = TaskAttemptId.parse(taO.getString("task-attempt-id"));
+ TaskAttempt ta = new TaskAttempt(taId, taO.getLong("start-time"),
+ taO.getLong("end-time"));
+ tcAttempts[k].tasks[l] = ta;
+ TaskId tid = taId.getTaskId();
+ ta.name = activityMap.get(tid.getActivityId());
+ ta.partition = tid.getPartition();
+ }
+ Arrays.sort(tcAttempts[k].tasks, new Comparator<TaskAttempt>() {
+ @Override
+ public int compare(TaskAttempt o1, TaskAttempt o2) {
+ return o1.startTime < o2.startTime ? -1
+ : (o1.startTime > o2.startTime ? 1 : 0);
+ }
+ });
+ }
+ }
+ Arrays.sort(tcAttempts, new Comparator<TaskClusterAttempt>() {
+ @Override
+ public int compare(TaskClusterAttempt o1, TaskClusterAttempt o2) {
+ return o1.startTime < o2.startTime ? -1 : (o1.startTime > o2.startTime ? 1 : 0);
+ }
+ });
+ tcList.add(tcAttempts);
+ }
+ }
+ }
+ }
+ }
+ }
+
+ Map<TaskAttemptId, TaskProfile> tpMap = new HashMap<TaskAttemptId, TaskProfile>();
if (jrO.has("profile")) {
JSONObject pO = jrO.getJSONObject("profile");
if (pO.has("joblets")) {
@@ -70,22 +160,27 @@
JSONArray tasksA = jobletO.getJSONArray("tasks");
for (int j = 0; j < tasksA.length(); ++j) {
JSONObject taskO = tasksA.getJSONObject(j);
- String activityId = taskO.getString("activity-id");
+ ActivityId activityId = ActivityId.parse(taskO.getString("activity-id"));
int partition = taskO.getInt("partition");
int attempt = taskO.getInt("attempt");
+ TaskAttemptId taId = new TaskAttemptId(new TaskId(activityId, partition), attempt);
if (taskO.has("partition-send-profile")) {
JSONArray taskProfilesA = taskO.getJSONArray("partition-send-profile");
for (int k = 0; k < taskProfilesA.length(); ++k) {
JSONObject ppO = taskProfilesA.getJSONObject(k);
long openTime = ppO.getLong("open-time");
long closeTime = ppO.getLong("close-time");
+ int resolution = ppO.getInt("resolution");
+ long offset = ppO.getLong("offset");
JSONArray frameTimesA = ppO.getJSONArray("frame-times");
long[] frameTimes = new long[frameTimesA.length()];
for (int l = 0; l < frameTimes.length; ++l) {
- frameTimes[l] = frameTimesA.getLong(l);
+ frameTimes[l] = frameTimesA.getInt(l) + offset;
}
- taskProfiles.add(new TaskProfile(activityId, partition, attempt, openTime,
- closeTime, frameTimes));
+ TaskProfile tp = new TaskProfile(taId, openTime, closeTime, frameTimes, resolution);
+ if (!tpMap.containsKey(tp.taId)) {
+ tpMap.put(tp.taId, tp);
+ }
}
}
}
@@ -93,68 +188,147 @@
}
}
}
- if (!taskProfiles.isEmpty()) {
- Collections.sort(taskProfiles, new Comparator<TaskProfile>() {
+
+ if (!tcList.isEmpty()) {
+ Collections.sort(tcList, new Comparator<TaskClusterAttempt[]>() {
@Override
- public int compare(TaskProfile o1, TaskProfile o2) {
- return o1.openTime < o2.openTime ? -1 : (o1.openTime > o2.openTime ? 1 : 0);
+ public int compare(TaskClusterAttempt[] o1, TaskClusterAttempt[] o2) {
+ if (o1.length == 0) {
+ return o2.length == 0 ? 0 : -1;
+ } else if (o2.length == 0) {
+ return 1;
+ }
+ return o1[0].startTime < o2[0].startTime ? -1 : (o1[0].startTime > o2[0].startTime ? 1 : 0);
}
});
- long startTime = taskProfiles.get(0).openTime;
- long timeRange = taskProfiles.get(taskProfiles.size() - 1).closeTime - startTime;
- int n = taskProfiles.size();
+ long range = maxTime - minTime;
+
+ double leftOffset = 20;
+
+ int xWidth = 1024;
+ double width = ((double) xWidth) / range;
StringBuilder buffer = new StringBuilder();
- buffer.append("<svg viewBox=\"0 0 ").append((timeRange + 1) * (WIDTH + 1))
- .append(' ').append((n + 1) * (HEIGHT + 1)).append("\" version=\"1.1\"\n");
- buffer.append("xmlns=\"http://www.w3.org/2000/svg\">\n");
- for (int i = 0; i < n; ++i) {
- TaskProfile tp = taskProfiles.get(i);
- open(buffer, i, tp.openTime - startTime);
- for (long ft : tp.frameTimes) {
- nextFrame(buffer, i, ft - startTime);
+ int y = 0;
+ for (TaskClusterAttempt[] tcAttempts : tcList) {
+ for (int i = 0; i < tcAttempts.length; ++i) {
+ TaskClusterAttempt tca = tcAttempts[i];
+ long startTime = tca.startTime - minTime;
+ long endTime = tca.endTime - minTime;
+ buffer.append("<rect x=\"").append(startTime * width + leftOffset).append("\" y=\"")
+ .append(y * (HEIGHT + 1)).append("\" width=\"").append(width * (endTime - startTime))
+ .append("\" height=\"").append(HEIGHT).append("\"/>\n");
+ buffer.append("<text x=\"").append(endTime * width + leftOffset + 20).append("\" y=\"")
+ .append(y * (HEIGHT + 1) + HEIGHT * 3 / 4).append("\">")
+ .append((endTime - startTime) + " ms").append("</text>\n");
+ ++y;
+ for (int j = 0; j < tca.tasks.length; ++j) {
+ TaskAttempt ta = tca.tasks[j];
+ long tStartTime = ta.startTime - minTime;
+ long tEndTime = ta.endTime - minTime;
+ buffer.append("<rect x=\"").append(tStartTime * width + leftOffset).append("\" y=\"")
+ .append(y * (HEIGHT + 1) + HEIGHT / 4).append("\" width=\"")
+ .append(width * (tEndTime - tStartTime)).append("\" height=\"").append(HEIGHT / 2)
+ .append("\" style=\"fill:rgb(255,255,255);stroke-width:1;stroke:rgb(0,0,0)\"/>\n");
+ buffer.append("<text x=\"").append(tEndTime * width + leftOffset + 20).append("\" y=\"")
+ .append(y * (HEIGHT + 1) + HEIGHT * 3 / 4).append("\">")
+ .append((tEndTime - tStartTime) + " ms (" + ta.name + ":" + ta.partition + ")")
+ .append("</text>\n");
+ TaskProfile tp = tpMap.get(ta.taId);
+ if (tp != null) {
+ for (int k = 0; k < tp.frameTimes.length; ++k) {
+ long taOpenTime = tp.openTime - minTime;
+ buffer.append("<rect x=\"")
+ .append(taOpenTime * width + leftOffset)
+ .append("\" y=\"")
+ .append(y * (HEIGHT + 1) + HEIGHT / 4)
+ .append("\" width=\"1\" height=\"")
+ .append(HEIGHT / 2)
+ .append("\" style=\"fill:rgb(255,0,0);stroke-width:1;stroke:rgb(255,0,0)\"/>\n");
+ for (int l = 0; l < tp.frameTimes.length; ++l) {
+ long ft = tp.frameTimes[l];
+ long ftn = l < tp.frameTimes.length - 1 ? tp.frameTimes[l + 1] : ft;
+ long taNextTime = ft - minTime;
+ long barWidth = ftn - ft;
+ buffer.append("<rect x=\"")
+ .append(taNextTime * width + leftOffset)
+ .append("\" y=\"")
+ .append(y * (HEIGHT + 1) + HEIGHT / 4)
+ .append("\" width=\"")
+ .append(barWidth == 0 ? 1 : (barWidth * width))
+ .append("\" height=\"")
+ .append(HEIGHT / 2)
+ .append("\" style=\"fill:rgb(0,255,0);stroke-width:1;stroke:rgb(0,255,0)\"/>\n");
+ }
+ long taCloseTime = tp.closeTime - minTime;
+ buffer.append("<rect x=\"")
+ .append(taCloseTime * width + leftOffset)
+ .append("\" y=\"")
+ .append(y * (HEIGHT + 1) + HEIGHT / 4)
+ .append("\" width=\"1\" height=\"")
+ .append(HEIGHT / 2)
+ .append("\" style=\"fill:rgb(0,0,255);stroke-width:1;stroke:rgb(0,0,255)\"/>\n");
+ }
+ }
+ ++y;
+ }
}
- close(buffer, i, tp.closeTime - startTime);
}
+ buffer.append("<rect x=\"").append(leftOffset).append("\" y=\"").append(0).append("\" width=\"").append(1)
+ .append("\" height=\"").append((y + 2) * (HEIGHT + 1)).append("\"/>\n");
+ buffer.append("<rect x=\"").append(0).append("\" y=\"").append((y + 1) * (HEIGHT + 1))
+ .append("\" width=\"").append(xWidth + 2 * leftOffset).append("\" height=\"").append(1)
+ .append("\"/>\n");
buffer.append("</svg>");
- Label markup = new Label("job-timeline", buffer.toString());
+ Label markup = new Label("job-timeline",
+ "<svg version=\"1.1\"\nxmlns=\"http://www.w3.org/2000/svg\" width=\"" + (xWidth * 1.5)
+ + "\" height=\"" + ((y + 2) * (HEIGHT + 1)) + "\">\n" + buffer.toString());
markup.setEscapeModelStrings(false);
add(markup);
- } else {
- add(new EmptyPanel("job-timeline"));
}
}
- private void open(StringBuilder buffer, int i, long openTime) {
- buffer.append("<rect x=\"").append(openTime * (WIDTH + 1)).append("\" y=\"").append(i * (HEIGHT + 1))
- .append("\" width=\"").append(WIDTH).append("\" height=\"").append(HEIGHT).append("\"/>\n");
+ private static class TaskAttempt {
+ private TaskAttemptId taId;
+ private long startTime;
+ private long endTime;
+ private String name;
+ private int partition;
+
+ public TaskAttempt(TaskAttemptId taId, long startTime, long endTime) {
+ this.taId = taId;
+ this.startTime = startTime;
+ this.endTime = endTime;
+ }
}
- private void close(StringBuilder buffer, int i, long closeTime) {
- buffer.append("<rect x=\"").append(closeTime * (WIDTH + 1)).append("\" y=\"").append(i * (HEIGHT + 1))
- .append("\" width=\"").append(WIDTH).append("\" height=\"").append(HEIGHT).append("\"/>\n");
- }
+ private static class TaskClusterAttempt {
+ private String tcId;
+ private int attempt;
+ private long startTime;
+ private long endTime;
+ private TaskAttempt[] tasks;
- private void nextFrame(StringBuilder buffer, int i, long frameTime) {
- buffer.append("<rect x=\"").append(frameTime * (WIDTH + 1)).append("\" y=\"").append(i * (HEIGHT + 1))
- .append("\" width=\"").append(WIDTH).append("\" height=\"").append(HEIGHT).append("\"/>\n");
+ public TaskClusterAttempt(String tcId, int attempt, long startTime, long endTime) {
+ this.tcId = tcId;
+ this.attempt = attempt;
+ this.startTime = startTime;
+ this.endTime = endTime;
+ }
}
private static class TaskProfile {
- private String activityId;
- private int partition;
- private int attempt;
+ private TaskAttemptId taId;
private long openTime;
private long closeTime;
private long[] frameTimes;
+ private int resolution;
- public TaskProfile(String activityId, int partition, int attempt, long openTime, long closeTime,
- long[] frameTimes) {
- this.activityId = activityId;
- this.partition = partition;
- this.activityId = activityId;
+ public TaskProfile(TaskAttemptId taId, long openTime, long closeTime, long[] frameTimes, int resolution) {
+ this.taId = taId;
this.openTime = openTime;
this.closeTime = closeTime;
this.frameTimes = frameTimes;
+ this.resolution = resolution;
}
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index df003b2..a72a866 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -316,6 +316,8 @@
JSONObject attempt = new JSONObject();
attempt.put("attempt", tca.getAttempt());
attempt.put("status", tca.getStatus());
+ attempt.put("start-time", tca.getStartTime());
+ attempt.put("end-time", tca.getEndTime());
JSONArray taskAttempts = new JSONArray();
for (TaskAttempt ta : tca.getTaskAttempts()) {
@@ -324,6 +326,8 @@
taskAttempt.put("task-attempt-id", ta.getTaskAttemptId());
taskAttempt.put("status", ta.getStatus());
taskAttempt.put("node-id", ta.getNodeId());
+ taskAttempt.put("start-time", ta.getStartTime());
+ taskAttempt.put("end-time", ta.getEndTime());
String failureDetails = ta.getFailureDetails();
if (failureDetails != null) {
taskAttempt.put("failure-details", failureDetails);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
index 43495db..7c0dd57 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -29,7 +29,7 @@
private final TaskAttemptId taskId;
- private final Task taskState;
+ private final Task task;
private String nodeId;
@@ -37,10 +37,16 @@
private String failureDetails;
- public TaskAttempt(TaskClusterAttempt tcAttempt, TaskAttemptId taskId, Task taskState) {
+ private long startTime;
+
+ private long endTime;
+
+ public TaskAttempt(TaskClusterAttempt tcAttempt, TaskAttemptId taskId, Task task) {
this.tcAttempt = tcAttempt;
this.taskId = taskId;
- this.taskState = taskState;
+ this.task = task;
+ startTime = -1;
+ endTime = -1;
}
public TaskClusterAttempt getTaskClusterAttempt() {
@@ -51,8 +57,8 @@
return taskId;
}
- public Task getTaskState() {
- return taskState;
+ public Task getTask() {
+ return task;
}
public String getNodeId() {
@@ -75,4 +81,20 @@
this.status = status;
this.failureDetails = details;
}
+
+ public long getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime(long startTime) {
+ this.startTime = startTime;
+ }
+
+ public long getEndTime() {
+ return endTime;
+ }
+
+ public void setEndTime(long endTime) {
+ this.endTime = endTime;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
index 9d33d12..84848bb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
@@ -32,9 +32,15 @@
private int pendingTaskCounter;
+ private long startTime;
+
+ private long endTime;
+
public TaskClusterAttempt(TaskCluster taskCluster, int attempt) {
this.taskCluster = taskCluster;
this.attempt = attempt;
+ startTime = -1;
+ endTime = -1;
}
public TaskCluster getTaskCluster() {
@@ -61,6 +67,22 @@
return status;
}
+ public long getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime(long startTime) {
+ this.startTime = startTime;
+ }
+
+ public long getEndTime() {
+ return endTime;
+ }
+
+ public void setEndTime(long endTime) {
+ this.endTime = endTime;
+ }
+
public void initializePendingTaskCounter() {
pendingTaskCounter = taskAttempts.length;
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
index afa1946..2dcbb3e 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -22,9 +22,10 @@
import java.util.Set;
import java.util.logging.Logger;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
import edu.uci.ics.hyracks.control.common.job.PartitionState;
@@ -52,7 +53,7 @@
while (i.hasNext()) {
PartitionRequest req = i.next();
if (partitionDescriptor.getState().isAtLeast(req.getMinimumState())) {
- matches.add(new Pair<PartitionDescriptor, PartitionRequest>(partitionDescriptor, req));
+ matches.add(Pair.<PartitionDescriptor, PartitionRequest> of(partitionDescriptor, req));
i.remove();
matched = true;
if (!partitionDescriptor.isReusable()) {
@@ -88,7 +89,7 @@
while (i.hasNext()) {
PartitionDescriptor descriptor = i.next();
if (descriptor.getState().isAtLeast(partitionRequest.getMinimumState())) {
- match = new Pair<PartitionDescriptor, PartitionRequest>(descriptor, partitionRequest);
+ match = Pair.<PartitionDescriptor, PartitionRequest> of(descriptor, partitionRequest);
if (!descriptor.isReusable()) {
i.remove();
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
index 90c6ad1..c36cf62 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
@@ -1,8 +1,9 @@
package edu.uci.ics.hyracks.control.cc.partitions;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.common.base.INodeController;
@@ -11,23 +12,14 @@
public class PartitionUtils {
public static void reportPartitionMatch(ClusterControllerService ccs, final PartitionId pid,
- Pair<PartitionDescriptor, PartitionRequest> match) {
- PartitionDescriptor desc = match.first;
- PartitionRequest req = match.second;
+ Pair<PartitionDescriptor, PartitionRequest> match) throws Exception {
+ PartitionDescriptor desc = match.getLeft();
+ PartitionRequest req = match.getRight();
NodeControllerState producerNCS = ccs.getNodeMap().get(desc.getNodeId());
NodeControllerState requestorNCS = ccs.getNodeMap().get(req.getNodeId());
final NetworkAddress dataport = producerNCS.getDataPort();
final INodeController requestorNC = requestorNCS.getNodeController();
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- try {
- requestorNC.reportPartitionAvailability(pid, dataport);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- });
+ requestorNC.reportPartitionAvailability(pid, dataport);
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
index 0e59c8df..f407ae3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
@@ -22,13 +22,14 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
import edu.uci.ics.hyracks.control.cc.job.ActivityClusterId;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -57,7 +58,7 @@
int producerOutputIndex = spec.getProducerOutputIndex(conn);
ActivityId inTask = jag.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
if (!eqSet.getActivities().contains(inTask)) {
- return new Pair<ActivityId, ActivityId>(t, inTask);
+ return Pair.<ActivityId, ActivityId> of(t, inTask);
}
}
}
@@ -70,7 +71,7 @@
int consumerInputIndex = spec.getConsumerInputIndex(conn);
ActivityId outTask = jag.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
if (!eqSet.getActivities().contains(outTask)) {
- return new Pair<ActivityId, ActivityId>(t, outTask);
+ return Pair.<ActivityId, ActivityId> of(t, outTask);
}
}
}
@@ -102,7 +103,7 @@
changed = false;
Pair<ActivityId, ActivityId> pair = findMergePair(jag, spec, stages);
if (pair != null) {
- merge(stageMap, stages, pair.first, pair.second);
+ merge(stageMap, stages, pair.getLeft(), pair.getRight());
changed = true;
}
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
index c721eca..929462b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -25,6 +25,8 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
@@ -38,7 +40,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
import edu.uci.ics.hyracks.control.cc.job.ActivityClusterPlan;
import edu.uci.ics.hyracks.control.cc.job.ActivityPlan;
@@ -128,7 +129,7 @@
Set<TaskId> cluster = taskClusterMap.get(ac1TaskStates[i].getTaskId());
for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
TaskId targetTID = ac2TaskStates[j].getTaskId();
- cInfoList.add(new Pair<TaskId, ConnectorDescriptorId>(targetTID, cdId));
+ cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
if (cPolicy.requiresProducerConsumerCoscheduling()) {
cluster.add(targetTID);
@@ -148,12 +149,13 @@
List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(tid);
if (cInfoList != null) {
for (Pair<TaskId, ConnectorDescriptorId> p : cInfoList) {
- Task targetTS = activityPlanMap.get(p.first.getActivityId()).getTasks()[p.first.getPartition()];
+ Task targetTS = activityPlanMap.get(p.getLeft().getActivityId()).getTasks()[p.getLeft()
+ .getPartition()];
TaskCluster targetTC = targetTS.getTaskCluster();
if (targetTC != tc) {
- ConnectorDescriptorId cdId = p.second;
- PartitionId pid = new PartitionId(jobRun.getJobId(), cdId, tid.getPartition(),
- p.first.getPartition());
+ ConnectorDescriptorId cdId = p.getRight();
+ PartitionId pid = new PartitionId(jobRun.getJobId(), cdId, tid.getPartition(), p.getLeft()
+ .getPartition());
tc.getProducedPartitions().add(pid);
targetTC.getRequiredPartitions().add(pid);
partitionProducingTaskClusterMap.put(pid, tc);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index 3feb5a6..e2c9ca3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -387,6 +387,7 @@
}
taskAttempt.setNodeId(nodeId);
taskAttempt.setStatus(TaskAttempt.TaskStatus.RUNNING, null);
+ taskAttempt.setStartTime(System.currentTimeMillis());
List<TaskAttemptDescriptor> tads = taskAttemptMap.get(nodeId);
if (tads == null) {
tads = new ArrayList<TaskAttemptDescriptor>();
@@ -399,6 +400,7 @@
tcAttempt.initializePendingTaskCounter();
tcAttempts.add(tcAttempt);
tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.RUNNING);
+ tcAttempt.setStartTime(System.currentTimeMillis());
inProgressTaskClusters.add(tc);
}
@@ -446,7 +448,7 @@
public void run() {
try {
node.getNodeController().startTasks(appName, jobId, JavaSerializationUtils.serialize(jag),
- taskDescriptors, connectorPolicies, null);
+ taskDescriptors, connectorPolicies);
} catch (IOException e) {
e.printStackTrace();
} catch (Exception e) {
@@ -459,10 +461,11 @@
}
private void abortJob(Exception exception) {
- for (TaskCluster tc : inProgressTaskClusters) {
+ Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<TaskCluster>(inProgressTaskClusters);
+ for (TaskCluster tc : inProgressTaskClustersCopy) {
abortTaskCluster(findLastTaskClusterAttempt(tc));
}
- inProgressTaskClusters.clear();
+ assert inProgressTaskClusters.isEmpty();
ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exception));
}
@@ -477,6 +480,7 @@
LOGGER.info("Checking " + taId + ": " + ta.getStatus());
if (status == TaskAttempt.TaskStatus.RUNNING || status == TaskAttempt.TaskStatus.COMPLETED) {
ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
+ ta.setEndTime(System.currentTimeMillis());
List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta.getNodeId());
if (abortTaskAttempts == null) {
abortTaskAttempts = new ArrayList<TaskAttemptId>();
@@ -524,6 +528,7 @@
TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
if (tca != null) {
abortTaskCluster(tca);
+ tca.setEndTime(System.currentTimeMillis());
tca.setStatus(TaskClusterAttempt.TaskClusterStatus.ABORTED);
}
}
@@ -571,14 +576,16 @@
public void notifyTaskComplete(TaskAttempt ta) throws HyracksException {
TaskAttemptId taId = ta.getTaskAttemptId();
- TaskCluster tc = ta.getTaskState().getTaskCluster();
+ TaskCluster tc = ta.getTask().getTaskCluster();
TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
TaskAttempt.TaskStatus taStatus = ta.getStatus();
if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
+ ta.setEndTime(System.currentTimeMillis());
if (lastAttempt.decrementPendingTasksCounter() == 0) {
lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
+ lastAttempt.setEndTime(System.currentTimeMillis());
inProgressTaskClusters.remove(tc);
startRunnableActivityClusters();
}
@@ -604,13 +611,14 @@
try {
LOGGER.info("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
TaskAttemptId taId = ta.getTaskAttemptId();
- TaskCluster tc = ta.getTaskState().getTaskCluster();
+ TaskCluster tc = ta.getTask().getTaskCluster();
TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
LOGGER.info("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
ta.setStatus(TaskAttempt.TaskStatus.FAILED, details);
abortTaskCluster(lastAttempt);
lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
+ lastAttempt.setEndTime(System.currentTimeMillis());
abortDoomedTaskClusters();
if (lastAttempt.getAttempt() >= ac.getMaxTaskClusterAttempts()) {
abortJob(new HyracksException(details));
@@ -648,6 +656,7 @@
assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
if (deadNodes.contains(ta.getNodeId())) {
ta.setStatus(TaskAttempt.TaskStatus.FAILED, "Node " + ta.getNodeId() + " failed");
+ ta.setEndTime(System.currentTimeMillis());
abort = true;
}
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 321b6b8..e30a718 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.control.cc.work;
import java.util.Set;
+import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -25,6 +26,8 @@
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
public class JobCleanupWork extends AbstractWork {
+ private static final Logger LOGGER = Logger.getLogger(JobCleanupWork.class.getName());
+
private ClusterControllerService ccs;
private JobId jobId;
private JobStatus status;
@@ -40,6 +43,14 @@
@Override
public void run() {
final JobRun run = ccs.getActiveRunMap().get(jobId);
+ if (run == null) {
+ LOGGER.warning("Unable to find JobRun with id: " + jobId);
+ return;
+ }
+ if (run.getPendingStatus() != null) {
+ LOGGER.warning("Ignoring duplicate cleanup for JobRun with id: " + jobId);
+ return;
+ }
Set<String> targetNodes = run.getParticipatingNodeIds();
run.getCleanupPendingNodeIds().addAll(targetNodes);
run.setPendingStatus(status, exception);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
index 87b5878..2a844ad 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
@@ -16,8 +16,9 @@
import java.util.List;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
@@ -46,7 +47,11 @@
List<Pair<PartitionDescriptor, PartitionRequest>> matches = pmm
.registerPartitionDescriptor(partitionDescriptor);
for (Pair<PartitionDescriptor, PartitionRequest> match : matches) {
- PartitionUtils.reportPartitionMatch(ccs, pid, match);
+ try {
+ PartitionUtils.reportPartitionMatch(ccs, pid, match);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
}
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
index 19716a4..a71875f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
@@ -14,8 +14,9 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
@@ -43,7 +44,11 @@
PartitionMatchMaker pmm = run.getPartitionMatchMaker();
Pair<PartitionDescriptor, PartitionRequest> match = pmm.matchPartitionRequest(partitionRequest);
if (match != null) {
- PartitionUtils.reportPartitionMatch(ccs, pid, match);
+ try {
+ PartitionUtils.reportPartitionMatch(ccs, pid, match);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
}
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
index 2838ed4..2694af4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
@@ -39,7 +39,7 @@
@Override
protected void performEvent(TaskAttempt ta) {
try {
- ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+ ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
JobRun run = ac.getJobRun();
if (statistics != null) {
JobProfile jobProfile = run.getJobProfile();
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
index b80d928..3201bec 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
@@ -31,7 +31,7 @@
@Override
protected void performEvent(TaskAttempt ta) {
- ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+ ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
ac.getJobRun().getScheduler().notifyTaskFailure(ta, ac, details);
}
diff --git a/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html b/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html
index f11e480..d2068fd 100644
--- a/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html
+++ b/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html
@@ -5,5 +5,5 @@
</div>
<div id="job-run" wicket:id="job-run" style="display: none;">
</div>
- <div wicket:id="job-timeline"></div>
+ <div wicket:id="job-timeline" style="overflow: auto;"></div>
</wicket:extend>
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
index cf64554..42d0ecb 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
@@ -14,14 +14,9 @@
*/
package edu.uci.ics.hyracks.control.common;
-import java.rmi.RemoteException;
-import java.rmi.server.UnicastRemoteObject;
-
import edu.uci.ics.hyracks.control.common.service.IService;
-public abstract class AbstractRemoteService extends UnicastRemoteObject implements IService {
- private static final long serialVersionUID = 1L;
-
- public AbstractRemoteService() throws RemoteException {
+public abstract class AbstractRemoteService implements IService {
+ public AbstractRemoteService() {
}
}
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 53003d0..40e9347 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.hyracks.control.common.base;
-import java.rmi.Remote;
import java.util.List;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
@@ -27,10 +26,10 @@
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
-public interface IClusterController extends Remote {
+public interface IClusterController {
public NodeParameters registerNode(NodeRegistration reg) throws Exception;
- public void unregisterNode(INodeController nodeController) throws Exception;
+ public void unregisterNode(String nodeId) throws Exception;
public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
throws Exception;
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
index fdf49e9..47f4ceb 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.hyracks.control.common.base;
-import java.rmi.Remote;
import java.util.List;
import java.util.Map;
@@ -25,23 +24,16 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
-public interface INodeController extends Remote {
- public String getId() throws Exception;
-
- public NCConfig getConfiguration() throws Exception;
-
+public interface INodeController {
public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
- Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, byte[] ctxVarBytes) throws Exception;
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) throws Exception;
public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception;
- public void notifyRegistration(IClusterController ccs) throws Exception;
-
public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
throws Exception;
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index 5091d12..4536fc4 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -19,8 +19,17 @@
import org.kohsuke.args4j.Option;
public class CCConfig {
- @Option(name = "-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
- public int port = 1099;
+ @Option(name = "-client-net-ip-address", usage = "Sets the IP Address to listen for connections from clients", required = true)
+ public String clientNetIpAddress;
+
+ @Option(name = "-client-net-port", usage = "Sets the port to listen for connections from clients (default 1098)")
+ public int clientNetPort = 1098;
+
+ @Option(name = "-cluster-net-ip-address", usage = "Sets the IP Address to listen for connections from ", required = true)
+ public String clusterNetIpAddress;
+
+ @Option(name = "-cluster-net-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
+ public int clusterNetPort = 1099;
@Option(name = "-http-port", usage = "Sets the http port for the Cluster Controller (default: 19001)")
public int httpPort = 19001;
@@ -44,8 +53,14 @@
public String ccRoot = "ClusterControllerService";
public void toCommandLine(List<String> cList) {
- cList.add("-port");
- cList.add(String.valueOf(port));
+ cList.add("-client-net-ip-address");
+ cList.add(clientNetIpAddress);
+ cList.add("-client-net-port");
+ cList.add(String.valueOf(clientNetPort));
+ cList.add("-cluster-net-ip-address");
+ cList.add(clusterNetIpAddress);
+ cList.add("-cluster-net-port");
+ cList.add(String.valueOf(clusterNetPort));
cList.add("-http-port");
cList.add(String.valueOf(httpPort));
cList.add("-heartbeat-period");
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 02ce0f8..c55f34a 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -22,16 +22,19 @@
public class NCConfig implements Serializable {
private static final long serialVersionUID = 1L;
- @Option(name = "-cc-host", usage = "Cluster Controller host name")
+ @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 = "-node-id", usage = "Logical name of node controller unique within the cluster")
+ @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 = "-data-ip-address", usage = "IP Address to bind data listener")
+ @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
public String dataIPAddress;
@Option(name = "-frame-size", usage = "Frame Size to use for data communication (default: 32768)")
@@ -54,6 +57,8 @@
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");
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
index ff92c2d..f6dde10 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -15,15 +15,15 @@
package edu.uci.ics.hyracks.control.common.controllers;
import java.io.Serializable;
+import java.net.InetSocketAddress;
import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatSchema;
public final class NodeRegistration implements Serializable {
private static final long serialVersionUID = 1L;
- private final INodeController nc;
+ private final InetSocketAddress ncAddress;
private final String nodeId;
@@ -41,9 +41,9 @@
private final HeartbeatSchema hbSchema;
- public NodeRegistration(INodeController nc, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
+ public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
String osName, String arch, String osVersion, int nProcessors, HeartbeatSchema hbSchema) {
- this.nc = nc;
+ this.ncAddress = ncAddress;
this.nodeId = nodeId;
this.ncConfig = ncConfig;
this.dataPort = dataPort;
@@ -54,8 +54,8 @@
this.hbSchema = hbSchema;
}
- public INodeController getNodeController() {
- return nc;
+ public InetSocketAddress getNodeControllerAddress() {
+ return ncAddress;
}
public String getNodeId() {
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
index 5457ac5..46a964a 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -3,6 +3,7 @@
import java.util.HashMap;
import java.util.Map;
+import org.json.JSONArray;
import org.json.JSONException;
import org.json.JSONObject;
@@ -32,12 +33,13 @@
public JSONObject toJSON() throws JSONException {
JSONObject json = new JSONObject();
- json.put("type", "job-profile");
json.put("job-id", jobId.toString());
populateCounters(json);
+ JSONArray jobletsArray = new JSONArray();
for (JobletProfile p : jobletProfiles.values()) {
- json.accumulate("joblets", p.toJSON());
+ jobletsArray.put(p.toJSON());
}
+ json.put("joblets", jobletsArray);
return json;
}
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
index 78f885d..0c60006 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
@@ -47,7 +47,6 @@
public JSONObject toJSON() throws JSONException {
JSONObject json = new JSONObject();
- json.put("type", "joblet-profile");
json.put("node-id", nodeId.toString());
populateCounters(json);
JSONArray tasks = new JSONArray();
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
index f6568d9..ef61796 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
@@ -17,6 +17,7 @@
import java.io.Serializable;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
public class PartitionProfile implements Serializable {
private static final long serialVersionUID = 1L;
@@ -27,13 +28,13 @@
private final long closeTime;
- private final byte[] frameTimes;
+ private final MultiResolutionEventProfiler mrep;
- public PartitionProfile(PartitionId pid, long openTime, long closeTime, byte[] frameTimes) {
+ public PartitionProfile(PartitionId pid, long openTime, long closeTime, MultiResolutionEventProfiler mrep) {
this.pid = pid;
this.openTime = openTime;
this.closeTime = closeTime;
- this.frameTimes = frameTimes;
+ this.mrep = mrep;
}
public PartitionId getPartitionId() {
@@ -48,7 +49,7 @@
return closeTime;
}
- public byte[] getFrameTimes() {
- return frameTimes;
+ public MultiResolutionEventProfiler getSamples() {
+ return mrep;
}
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
index 3764a01..2116f61 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.hyracks.control.common.job.profiling.om;
-import java.io.ByteArrayInputStream;
import java.util.Map;
import org.json.JSONArray;
@@ -23,6 +22,7 @@
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
public class TaskProfile extends AbstractProfile {
private static final long serialVersionUID = 1L;
@@ -48,7 +48,6 @@
public JSONObject toJSON() throws JSONException {
JSONObject json = new JSONObject();
- json.put("type", "task-profile");
json.put("activity-id", taskAttemptId.getTaskId().getActivityId().toString());
json.put("partition", taskAttemptId.getTaskId().getPartition());
json.put("attempt", taskAttemptId.getAttempt());
@@ -65,24 +64,17 @@
ppObj.put("partition-id", pidObj);
ppObj.put("open-time", pp.getOpenTime());
ppObj.put("close-time", pp.getCloseTime());
- JSONArray ftArray = new JSONArray();
- byte[] ftb = pp.getFrameTimes();
- ByteArrayInputStream bais = new ByteArrayInputStream(ftb);
- long value = 0;
- int vLen = 0;
- long time = pp.getOpenTime();
- for (int i = 0; i < ftb.length; ++i) {
- byte b = (byte) bais.read();
- ++vLen;
- value += (((long) (b & 0xef)) << ((vLen - 1) * 7));
- if ((b & 0x80) == 0) {
- time += value;
- ftArray.put(time);
- vLen = 0;
- value = 0;
- }
+ MultiResolutionEventProfiler samples = pp.getSamples();
+ ppObj.put("offset", samples.getOffset());
+ int resolution = samples.getResolution();
+ int sampleCount = samples.getCount();
+ JSONArray ftA = new JSONArray();
+ int[] ft = samples.getSamples();
+ for (int i = 0; i < sampleCount; ++i) {
+ ftA.put(ft[i]);
}
- ppObj.put("frame-times", ftArray);
+ ppObj.put("frame-times", ftA);
+ ppObj.put("resolution", resolution);
pspArray.put(ppObj);
}
json.put("partition-send-profile", pspArray);
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
index f5ae1f4..36f0c49 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
@@ -16,6 +16,8 @@
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
@@ -27,15 +29,21 @@
private final WorkerThread thread;
private final Semaphore stopSemaphore;
private boolean stopped;
+ private final AtomicInteger enqueueCount;
+ private final AtomicInteger dequeueCount;
public WorkQueue() {
queue = new LinkedBlockingQueue<AbstractWork>();
thread = new WorkerThread();
stopSemaphore = new Semaphore(1);
+ enqueueCount = new AtomicInteger();
+ dequeueCount = new AtomicInteger();
}
public void start() throws HyracksException {
stopped = false;
+ enqueueCount.set(0);
+ dequeueCount.set(0);
try {
stopSemaphore.acquire();
} catch (InterruptedException e) {
@@ -61,6 +69,10 @@
}
public void schedule(AbstractWork event) {
+ enqueueCount.incrementAndGet();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Enqueue: " + enqueueCount);
+ }
if (LOGGER.isLoggable(event.logLevel())) {
LOGGER.log(event.logLevel(), "Scheduling: " + event);
}
@@ -80,7 +92,7 @@
@Override
public void run() {
try {
- Runnable r;
+ AbstractWork r;
while (true) {
synchronized (WorkQueue.this) {
if (stopped) {
@@ -92,7 +104,14 @@
} catch (InterruptedException e) {
continue;
}
+ dequeueCount.incrementAndGet();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Dequeue: " + dequeueCount + "/" + enqueueCount);
+ }
try {
+ if (LOGGER.isLoggable(r.logLevel())) {
+ LOGGER.log(r.logLevel(), "Executing: " + r);
+ }
r.run();
} catch (Exception e) {
e.printStackTrace();
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 1a98ebd..07d8ad7 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -145,7 +145,7 @@
}
public synchronized void notifyTaskComplete(Task task) throws Exception {
- taskMap.remove(task);
+ taskMap.remove(task.getTaskAttemptId());
try {
TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), task.getPartitionSendProfile());
task.dumpProfile(taskProfile);
@@ -159,7 +159,7 @@
}
public synchronized void notifyTaskFailed(Task task, String details) throws Exception {
- taskMap.remove(task);
+ taskMap.remove(task.getTaskAttemptId());
try {
nodeController.getClusterController().notifyTaskFailure(jobId, task.getTaskAttemptId(),
nodeController.getId(), details);
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 3fd10a8..d869515 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -23,8 +23,7 @@
import java.lang.management.RuntimeMXBean;
import java.lang.management.ThreadMXBean;
import java.net.InetAddress;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
+import java.net.InetSocketAddress;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Hashtable;
@@ -58,6 +57,8 @@
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatSchema;
+import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerRemoteProxy;
+import edu.uci.ics.hyracks.control.common.ipc.NodeControllerDelegateIPCI;
import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
import edu.uci.ics.hyracks.control.common.work.FutureValue;
@@ -74,18 +75,20 @@
import edu.uci.ics.hyracks.control.nc.work.DestroyApplicationWork;
import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
public class NodeControllerService extends AbstractRemoteService implements INodeController {
private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
- private static final long serialVersionUID = 1L;
-
private NCConfig ncConfig;
private final String id;
private final IHyracksRootContext ctx;
+ private final IPCSystem ipc;
+
private final PartitionManager partitionManager;
private final ConnectionManager connectionManager;
@@ -122,6 +125,8 @@
this.ncConfig = ncConfig;
id = ncConfig.nodeId;
executor = Executors.newCachedThreadPool();
+ NodeControllerDelegateIPCI ipci = new NodeControllerDelegateIPCI(this);
+ ipc = new IPCSystem(new InetSocketAddress(ncConfig.clusterNetIPAddress, 0), ipci, executor);
this.ctx = new RootHyracksContext(ncConfig.frameSize, new IOManager(getDevices(ncConfig.ioDevices), executor));
if (id == null) {
throw new Exception("id not set");
@@ -160,27 +165,28 @@
@Override
public void start() throws Exception {
LOGGER.log(Level.INFO, "Starting NodeControllerService");
+ ipc.start();
connectionManager.start();
- Registry registry = LocateRegistry.getRegistry(ncConfig.ccHost, ncConfig.ccPort);
- IClusterController cc = (IClusterController) registry.lookup(IClusterController.class.getName());
+ IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
+ this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
for (int i = 0; i < gcInfos.length; ++i) {
gcInfos[i] = new HeartbeatSchema.GarbageCollectorInfo(gcMXBeans.get(i).getName());
}
HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
- this.nodeParameters = cc.registerNode(new NodeRegistration(this, id, ncConfig, connectionManager
- .getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean
- .getAvailableProcessors(), hbSchema));
+ this.nodeParameters = ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig,
+ connectionManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(),
+ osMXBean.getAvailableProcessors(), hbSchema));
queue.start();
- heartbeatTask = new HeartbeatTask(cc);
+ heartbeatTask = new HeartbeatTask(ccs);
// Schedule heartbeat generator.
timer.schedule(heartbeatTask, 0, nodeParameters.getHeartbeatPeriod());
if (nodeParameters.getProfileDumpPeriod() > 0) {
// Schedule profile dump generator.
- timer.schedule(new ProfileDumpTask(cc), 0, nodeParameters.getProfileDumpPeriod());
+ timer.schedule(new ProfileDumpTask(ccs), 0, nodeParameters.getProfileDumpPeriod());
}
LOGGER.log(Level.INFO, "Started NodeControllerService");
@@ -196,7 +202,6 @@
LOGGER.log(Level.INFO, "Stopped NodeControllerService");
}
- @Override
public String getId() {
return id;
}
@@ -236,23 +241,17 @@
@Override
public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
List<TaskAttemptDescriptor> taskDescriptors,
- Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, byte[] ctxVarBytes) throws Exception {
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap) throws Exception {
StartTasksWork stw = new StartTasksWork(this, appName, jobId, jagBytes, taskDescriptors, connectorPoliciesMap);
- queue.scheduleAndSync(stw);
+ queue.schedule(stw);
}
@Override
public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception {
CleanupJobletWork cjw = new CleanupJobletWork(this, jobId, status);
- queue.scheduleAndSync(cjw);
+ queue.schedule(cjw);
}
- @Override
- public void notifyRegistration(IClusterController ccs) throws Exception {
- this.ccs = ccs;
- }
-
- @Override
public NCConfig getConfiguration() throws Exception {
return ncConfig;
}
@@ -260,20 +259,24 @@
@Override
public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
AbortTasksWork atw = new AbortTasksWork(this, jobId, tasks);
- queue.scheduleAndSync(atw);
+ queue.schedule(atw);
}
@Override
public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
throws Exception {
- CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState);
- queue.scheduleAndSync(caw);
+ FutureValue<Object> fv = new FutureValue<Object>();
+ CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState, fv);
+ queue.schedule(caw);
+ fv.get();
}
@Override
public void destroyApplication(String appName) throws Exception {
- DestroyApplicationWork daw = new DestroyApplicationWork(this, appName);
- queue.scheduleAndSync(daw);
+ FutureValue<Object> fv = new FutureValue<Object>();
+ DestroyApplicationWork daw = new DestroyApplicationWork(this, appName, fv);
+ queue.schedule(daw);
+ fv.get();
}
@Override
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
index b9eaf1f..6e38ef7 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
@@ -63,7 +63,8 @@
this.ctx = ctx;
serverChannel = ServerSocketChannel.open();
ServerSocket serverSocket = serverChannel.socket();
- serverSocket.bind(new InetSocketAddress(inetAddress, 0));
+ serverSocket.bind(new InetSocketAddress(inetAddress, 0), 0);
+ serverSocket.setReuseAddress(true);
stopped = false;
connectionListener = new ConnectionListenerThread();
dataListener = new DataListenerThread();
@@ -97,6 +98,7 @@
public ConnectionListenerThread() {
super("Hyracks NC Connection Listener");
setDaemon(true);
+ setPriority(MAX_PRIORITY);
}
@Override
@@ -159,6 +161,7 @@
if (!pendingIncomingConnections.isEmpty()) {
for (SocketChannel sc : pendingIncomingConnections) {
sc.configureBlocking(false);
+ sc.socket().setReuseAddress(true);
SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
ByteBuffer buffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
scKey.attach(buffer);
@@ -170,6 +173,7 @@
for (INetworkChannel nc : pendingOutgoingConnections) {
SocketChannel sc = SocketChannel.open();
sc.configureBlocking(false);
+ sc.socket().setReuseAddress(true);
SelectionKey scKey = sc.register(selector, 0);
scKey.attach(nc);
nc.setSelectionKey(scKey);
@@ -202,6 +206,7 @@
} catch (HyracksException e) {
key.cancel();
sc.close();
+ channel.abort();
}
} else {
buffer.compact();
@@ -210,15 +215,19 @@
} else {
INetworkChannel channel = (INetworkChannel) key.attachment();
boolean close = false;
+ boolean error = false;
try {
close = channel.dispatchNetworkEvent();
} catch (IOException e) {
e.printStackTrace();
- close = true;
+ error = true;
}
- if (close) {
+ if (close || error) {
key.cancel();
sc.close();
+ if (error) {
+ channel.abort();
+ }
}
}
}
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
index 58f7088..23cf514 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
@@ -130,7 +130,7 @@
public synchronized boolean dispatchNetworkEvent() throws IOException {
if (aborted) {
eos = true;
- monitor.notifyEndOfStream(this);
+ monitor.notifyFailure(this);
return true;
}
if (key.isConnectable()) {
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
index ea4dbd6..affa01c 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.hyracks.control.nc.profiling;
-import java.io.ByteArrayOutputStream;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -23,10 +22,13 @@
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
import edu.uci.ics.hyracks.control.common.job.profiling.om.PartitionProfile;
import edu.uci.ics.hyracks.control.nc.Task;
public class ProfilingPartitionWriterFactory implements IPartitionWriterFactory {
+ private static final int N_SAMPLES = 64;
+
private final IHyracksTaskContext ctx;
private final IConnectorDescriptor cd;
@@ -52,31 +54,17 @@
private long closeTime;
- private long prevTime;
-
- private ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ MultiResolutionEventProfiler mrep = new MultiResolutionEventProfiler(N_SAMPLES);
@Override
public void open() throws HyracksDataException {
- baos.reset();
openTime = System.currentTimeMillis();
- prevTime = openTime;
writer.open();
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- long time = System.currentTimeMillis();
- long diff = time - prevTime;
- prevTime = time;
- do {
- byte b = (byte) (diff & 0x7f);
- diff >>= 7;
- if (diff != 0) {
- b |= 0x80;
- }
- baos.write(b);
- } while (diff != 0);
+ mrep.reportEvent();
writer.nextFrame(buffer);
}
@@ -89,8 +77,7 @@
public void close() throws HyracksDataException {
closeTime = System.currentTimeMillis();
((Task) ctx).setPartitionSendProfile(new PartitionProfile(new PartitionId(ctx.getJobletContext()
- .getJobId(), cd.getConnectorId(), senderIndex, receiverIndex), openTime, closeTime, baos
- .toByteArray()));
+ .getJobId(), cd.getConnectorId(), senderIndex, receiverIndex), openTime, closeTime, mrep));
writer.close();
}
};
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java
index 4e51142..7bebb53 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java
@@ -46,10 +46,19 @@
}
ncs.getPartitionManager().unregisterPartitions(jobId);
Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
- Joblet joblet = jobletMap.get(jobId);
+ Joblet joblet = jobletMap.remove(jobId);
if (joblet != null) {
joblet.cleanup(status);
}
- ncs.getClusterController().notifyJobletCleanup(jobId, ncs.getId());
+ ncs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ ncs.getClusterController().notifyJobletCleanup(jobId, ncs.getId());
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ });
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
index 120e376..5b60a08 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
@@ -29,6 +29,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
@@ -44,41 +45,49 @@
private final byte[] serializedDistributedState;
+ private final FutureValue<Object> fv;
+
public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
- byte[] serializedDistributedState) {
+ byte[] serializedDistributedState, FutureValue<Object> fv) {
this.ncs = ncs;
this.appName = appName;
this.deployHar = deployHar;
this.serializedDistributedState = serializedDistributedState;
+ this.fv = fv;
}
@Override
protected void doRun() throws Exception {
- NCApplicationContext appCtx;
- Map<String, NCApplicationContext> applications = ncs.getApplications();
- if (applications.containsKey(appName)) {
- throw new HyracksException("Duplicate application with name: " + appName + " being created.");
- }
- appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
- applications.put(appName, appCtx);
- if (deployHar) {
- NCConfig ncConfig = ncs.getConfiguration();
- NodeParameters nodeParameters = ncs.getNodeParameters();
- HttpClient hc = new DefaultHttpClient();
- HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
- + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
- HttpResponse response = hc.execute(get);
- InputStream is = response.getEntity().getContent();
- OutputStream os = appCtx.getHarOutputStream();
- try {
- IOUtils.copyLarge(is, os);
- } finally {
- os.close();
- is.close();
+ try {
+ NCApplicationContext appCtx;
+ Map<String, NCApplicationContext> applications = ncs.getApplications();
+ if (applications.containsKey(appName)) {
+ throw new HyracksException("Duplicate application with name: " + appName + " being created.");
}
+ appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
+ applications.put(appName, appCtx);
+ if (deployHar) {
+ NCConfig ncConfig = ncs.getConfiguration();
+ NodeParameters nodeParameters = ncs.getNodeParameters();
+ HttpClient hc = new DefaultHttpClient();
+ HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
+ + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
+ HttpResponse response = hc.execute(get);
+ InputStream is = response.getEntity().getContent();
+ OutputStream os = appCtx.getHarOutputStream();
+ try {
+ IOUtils.copyLarge(is, os);
+ } finally {
+ os.close();
+ is.close();
+ }
+ }
+ appCtx.initializeClassPath();
+ appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
+ appCtx.initialize();
+ fv.setValue(null);
+ } catch (Exception e) {
+ fv.setException(e);
}
- appCtx.initializeClassPath();
- appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
- appCtx.initialize();
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
index 6a5fbfc..8ac0b5c 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
@@ -18,6 +18,7 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
@@ -29,17 +30,25 @@
private final String appName;
- public DestroyApplicationWork(NodeControllerService ncs, String appName) {
+ private FutureValue<Object> fv;
+
+ public DestroyApplicationWork(NodeControllerService ncs, String appName, FutureValue<Object> fv) {
this.ncs = ncs;
this.appName = appName;
+ this.fv = fv;
}
@Override
protected void doRun() throws Exception {
- Map<String, NCApplicationContext> applications = ncs.getApplications();
- ApplicationContext appCtx = applications.remove(appName);
- if (appCtx != null) {
- appCtx.deinitialize();
+ try {
+ Map<String, NCApplicationContext> applications = ncs.getApplications();
+ ApplicationContext appCtx = applications.remove(appName);
+ if (appCtx != null) {
+ appCtx.deinitialize();
+ }
+ fv.setValue(null);
+ } catch (Exception e) {
+ fv.setException(e);
}
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/pom.xml b/hyracks-dataflow-common/pom.xml
index 50dc4d2..142a91e 100644
--- a/hyracks-dataflow-common/pom.xml
+++ b/hyracks-dataflow-common/pom.xml
@@ -1,9 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-dataflow-common</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks</artifactId>
@@ -31,5 +28,10 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
index 61638d1..beaf862 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
@@ -62,7 +62,7 @@
public Object[] deserializeRecord() throws HyracksDataException {
int start = frameTupleAccessor.getTupleStartOffset(tIndex) + frameTupleAccessor.getFieldSlotsLength();
bbis.setByteBuffer(buffer, start);
- Object[] record = new Object[recordDescriptor.getFields().length];
+ Object[] record = new Object[recordDescriptor.getFieldCount()];
for (int i = 0; i < record.length; ++i) {
Object instance = recordDescriptor.getFields()[i].deserialize(di);
if (LOGGER.isLoggable(Level.FINEST)) {
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index 8ab5d4f..6c51d14 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -24,11 +24,12 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
/**
- * FrameTupleCursor is used to navigate over tuples in a Frame.
- * A frame is formatted with tuple data concatenated starting at offset 0, one tuple after another.
- * Offset FS - 4 holds an int indicating the number of tuples (N) in the frame. FS - ((i + 1) * 4) for i from
- * 0 to N - 1 holds an int indicating the offset of the (i + 1)^th tuple.
- * Every tuple is organized as a sequence of ints indicating the end of each field in the tuple relative to the end of the
+ * FrameTupleCursor is used to navigate over tuples in a Frame. A frame is
+ * formatted with tuple data concatenated starting at offset 0, one tuple after
+ * another. Offset FS - 4 holds an int indicating the number of tuples (N) in
+ * the frame. FS - ((i + 1) * 4) for i from 0 to N - 1 holds an int indicating
+ * the offset of the (i + 1)^th tuple. Every tuple is organized as a sequence of
+ * ints indicating the end of each field in the tuple relative to the end of the
* field slots.
*
* @author vinayakb
@@ -86,7 +87,7 @@
@Override
public int getFieldSlotsLength() {
- return recordDescriptor.getFields().length * 4;
+ return getFieldCount() * 4;
}
public void prettyPrint() {
@@ -96,7 +97,7 @@
System.err.println("TC: " + tc);
for (int i = 0; i < tc; ++i) {
System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
- for (int j = 0; j < recordDescriptor.getFields().length; ++j) {
+ for (int j = 0; j < getFieldCount(); ++j) {
System.err.print(j + ":(" + getFieldStartOffset(i, j) + ", " + getFieldEndOffset(i, j) + ") ");
System.err.print("{");
bbis.setByteBuffer(buffer, getTupleStartOffset(i) + getFieldSlotsLength() + getFieldStartOffset(i, j));
@@ -113,6 +114,6 @@
@Override
public int getFieldCount() {
- return recordDescriptor.getFields().length;
+ return recordDescriptor.getFieldCount();
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index e8f43cd..cbe2ae6 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -41,7 +41,7 @@
public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
buffer = ctx.allocateFrame();
- tb = new ArrayTupleBuilder(recordDescriptor.getFields().length);
+ tb = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
this.recordDescriptor = recordDescriptor;
this.frameWriter = frameWriter;
tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java
index 53b8d23..60fd395 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java
@@ -16,7 +16,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
public class UTF8StringNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
private static final long serialVersionUID = 1L;
@@ -26,14 +26,14 @@
return new INormalizedKeyComputer() {
@Override
public int normalize(byte[] bytes, int start, int length) {
- int len = StringUtils.getUTFLen(bytes, start);
+ int len = UTF8StringPointable.getUTFLen(bytes, start);
int nk = 0;
int offset = start + 2;
for (int i = 0; i < 2; ++i) {
nk <<= 16;
if (i < len) {
- nk += ((int) StringUtils.charAt(bytes, offset)) & 0xffff;
- offset += StringUtils.charSize(bytes, offset);
+ nk += ((int) UTF8StringPointable.charAt(bytes, offset)) & 0xffff;
+ offset += UTF8StringPointable.charSize(bytes, offset);
}
}
return nk;
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
index 8d69b95..7769539 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
@@ -39,6 +39,9 @@
return new ITuplePartitionComputer() {
@Override
public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) {
+ if (nParts == 1) {
+ return 0;
+ }
int h = 0;
int startOffset = accessor.getTupleStartOffset(tIndex);
int slotLength = accessor.getFieldSlotsLength();
@@ -49,7 +52,7 @@
int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
int fh = hashFn
.hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
- h += fh;
+ h = h * 31 + fh;
}
if (h < 0) {
h = -h;
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
index bcaeb22..a29209c 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
@@ -18,79 +18,6 @@
import java.io.IOException;
public class StringUtils {
- public static char charAt(byte[] b, int s) {
- int c = b[s] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- return (char) c;
-
- case 12:
- case 13:
- return (char) (((c & 0x1F) << 6) | ((b[s + 1]) & 0x3F));
-
- case 14:
- return (char) (((c & 0x0F) << 12) | (((b[s + 1]) & 0x3F) << 6) | (((b[s + 2]) & 0x3F) << 0));
-
- default:
- throw new IllegalArgumentException();
- }
- }
-
- public static int charSize(byte[] b, int s) {
- int c = b[s] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- return 1;
-
- case 12:
- case 13:
- return 2;
-
- case 14:
- return 3;
- }
- throw new IllegalStateException();
- }
-
- public static int getModifiedUTF8Len(char c) {
- if (c >= 0x0000 && c <= 0x007F) {
- return 1;
- } else if (c <= 0x07FF) {
- return 2;
- } else {
- return 3;
- }
- }
-
- public static int getStrLen(byte[] b, int s) {
- int pos = s + 2;
- int end = pos + getUTFLen(b, s);
- int charCount = 0;
- while (pos < end) {
- charCount++;
- pos += charSize(b, pos);
- }
- return charCount;
- }
-
- public static int getUTFLen(byte[] b, int s) {
- return ((b[s] & 0xff) << 8) + ((b[s + 1] & 0xff) << 0);
- }
-
public static void writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
if (c >= 0x0000 && c <= 0x007F) {
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
index d690279..87d9b35 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
@@ -18,11 +18,14 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.FloatBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
@@ -32,31 +35,31 @@
@SuppressWarnings("rawtypes")
public class SerdeUtils {
- public static ITypeTrait[] serdesToTypeTraits(ISerializerDeserializer[] serdes, int numSerdes) {
- ITypeTrait[] typeTraits = new ITypeTrait[numSerdes];
+ public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes, int numSerdes) {
+ ITypeTraits[] typeTraits = new ITypeTraits[numSerdes];
for (int i = 0; i < numSerdes; i++) {
typeTraits[i] = serdeToTypeTrait(serdes[i]);
}
return typeTraits;
}
- public static ITypeTrait serdeToTypeTrait(ISerializerDeserializer serde) {
+ public static ITypeTraits serdeToTypeTrait(ISerializerDeserializer serde) {
if (serde instanceof IntegerSerializerDeserializer) {
- return ITypeTrait.INTEGER_TYPE_TRAIT;
+ return IntegerPointable.TYPE_TRAITS;
}
if (serde instanceof Integer64SerializerDeserializer) {
- return ITypeTrait.INTEGER64_TYPE_TRAIT;
+ return LongPointable.TYPE_TRAITS;
}
if (serde instanceof FloatSerializerDeserializer) {
- return ITypeTrait.FLOAT_TYPE_TRAIT;
+ return FloatPointable.TYPE_TRAITS;
}
if (serde instanceof DoubleSerializerDeserializer) {
- return ITypeTrait.DOUBLE_TYPE_TRAIT;
+ return DoublePointable.TYPE_TRAITS;
}
if (serde instanceof BooleanSerializerDeserializer) {
- return ITypeTrait.BOOLEAN_TYPE_TRAIT;
+ return BooleanPointable.TYPE_TRAITS;
}
- return ITypeTrait.VARLEN_TYPE_TRAIT;
+ return UTF8StringPointable.TYPE_TRAITS;
}
public static IBinaryComparator[] serdesToComparators(ISerializerDeserializer[] serdes, int numSerdes) {
@@ -68,27 +71,10 @@
}
public static IBinaryComparator serdeToComparator(ISerializerDeserializer serde) {
- if (serde instanceof IntegerSerializerDeserializer) {
- return IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- }
- if (serde instanceof Integer64SerializerDeserializer) {
- throw new UnsupportedOperationException("Binary comparator for Integer64 not implemented.");
- }
- if (serde instanceof FloatSerializerDeserializer) {
- return FloatBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- }
- if (serde instanceof DoubleSerializerDeserializer) {
- return DoubleBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- }
- if (serde instanceof BooleanSerializerDeserializer) {
- throw new UnsupportedOperationException("Binary comparator for Boolean not implemented.");
- }
- if (serde instanceof UTF8StringSerializerDeserializer) {
- return UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- }
- throw new UnsupportedOperationException("Binary comparator for + " + serde.toString() + " not implemented.");
+ IBinaryComparatorFactory f = serdeToComparatorFactory(serde);
+ return f.createBinaryComparator();
}
-
+
public static IBinaryComparatorFactory[] serdesToComparatorFactories(ISerializerDeserializer[] serdes, int numSerdes) {
IBinaryComparatorFactory[] comparatorsFactories = new IBinaryComparatorFactory[numSerdes];
for (int i = 0; i < numSerdes; i++) {
@@ -99,22 +85,22 @@
public static IBinaryComparatorFactory serdeToComparatorFactory(ISerializerDeserializer serde) {
if (serde instanceof IntegerSerializerDeserializer) {
- return IntegerBinaryComparatorFactory.INSTANCE;
+ return PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
}
if (serde instanceof Integer64SerializerDeserializer) {
- throw new UnsupportedOperationException("Binary comparator factory for Integer64 not implemented.");
+ return PointableBinaryComparatorFactory.of(LongPointable.FACTORY);
}
if (serde instanceof FloatSerializerDeserializer) {
- return FloatBinaryComparatorFactory.INSTANCE;
+ return PointableBinaryComparatorFactory.of(FloatPointable.FACTORY);
}
if (serde instanceof DoubleSerializerDeserializer) {
- return DoubleBinaryComparatorFactory.INSTANCE;
+ return PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
}
if (serde instanceof BooleanSerializerDeserializer) {
throw new UnsupportedOperationException("Binary comparator factory for Boolean not implemented.");
}
if (serde instanceof UTF8StringSerializerDeserializer) {
- return UTF8StringBinaryComparatorFactory.INSTANCE;
+ return PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
}
throw new UnsupportedOperationException("Binary comparator for + " + serde.toString() + " not implemented.");
}
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index ef2b4cb..2841509 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -142,8 +142,8 @@
@SuppressWarnings("deprecation")
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IRecordDescriptorProvider recordDescProvider,
- final int partition, int nPartitions)
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
return new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
@@ -191,7 +191,7 @@
RecordDescriptor outputRecordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
(Class<? extends Writable>) hadoopRecordReader.createKey().getClass(),
(Class<? extends Writable>) hadoopRecordReader.createValue().getClass());
- int nFields = outputRecordDescriptor.getFields().length;
+ int nFields = outputRecordDescriptor.getFieldCount();
ArrayTupleBuilder tb = new ArrayTupleBuilder(nFields);
writer.open();
try {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index f3535f8..4ffc060 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -41,7 +41,6 @@
public JSONObject toJSON() throws JSONException {
JSONObject jconn = new JSONObject();
- jconn.put("type", "connector");
jconn.put("id", getConnectorId().getId());
jconn.put("java-class", getClass().getName());
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index 15fc75d..1bb13ca 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -75,7 +75,6 @@
@Override
public JSONObject toJSON() throws JSONException {
JSONObject jop = new JSONObject();
- jop.put("type", "operator");
jop.put("id", getOperatorId().getId());
jop.put("java-class", getClass().getName());
jop.put("in-arity", getInputArity());
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
index 33819a8..2b37c7b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
@@ -172,6 +172,7 @@
PartitionId pid = (PartitionId) channel.getAttachment();
int senderIndex = pid.getSenderIndex();
failSenders.set(senderIndex);
+ eosSenders.set(senderIndex);
NonDeterministicPartitionCollector.this.notifyAll();
}
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
index 5bc6ee5..8544891 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
@@ -75,7 +75,7 @@
writer.open();
try {
while (true) {
- Object[] record = new Object[desc.getFields().length];
+ Object[] record = new Object[desc.getFieldCount()];
if (!reader.read(record)) {
break;
}
@@ -107,8 +107,8 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new DeserializedFileScanOperator(partition), null);
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index e0f5a65..47a3158 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -60,8 +60,8 @@
* @see edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.uci.ics.hyracks.api.context.IHyracksContext, edu.uci.ics.hyracks.api.job.IOperatorEnvironment, edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int, int)
*/
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
- final int partition, int nPartitions)
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
// Output files
final FileSplit[] splits = fileSplitProvider.getFileSplits();
@@ -96,7 +96,7 @@
int start = frameTupleAccessor.getTupleStartOffset(tIndex)
+ frameTupleAccessor.getFieldSlotsLength();
bbis.setByteBuffer(buffer, start);
- Object[] record = new Object[recordDescriptor.getFields().length];
+ Object[] record = new Object[recordDescriptor.getFieldCount()];
for (int i = 0; i < record.length; ++i) {
Object instance = recordDescriptor.getFields()[i].deserialize(di);
if (i == 0) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
index f4cfe98..ef427d1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
@@ -34,223 +34,242 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
class GroupingHashTable {
- /**
- * The pointers in the link store 3 int values for each entry in the
- * hashtable: (bufferIdx, tIndex, accumulatorIdx).
- *
- * @author vinayakb
- */
- private static class Link {
- private static final int INIT_POINTERS_SIZE = 9;
+ /**
+ * The pointers in the link store 3 int values for each entry in the
+ * hashtable: (bufferIdx, tIndex, accumulatorIdx).
+ *
+ * @author vinayakb
+ */
+ private static class Link {
+ private static final int INIT_POINTERS_SIZE = 9;
- int[] pointers;
- int size;
+ int[] pointers;
+ int size;
- Link() {
- pointers = new int[INIT_POINTERS_SIZE];
- size = 0;
- }
+ Link() {
+ pointers = new int[INIT_POINTERS_SIZE];
+ size = 0;
+ }
- void add(int bufferIdx, int tIndex, int accumulatorIdx) {
- while (size + 3 > pointers.length) {
- pointers = Arrays.copyOf(pointers, pointers.length * 2);
- }
- pointers[size++] = bufferIdx;
- pointers[size++] = tIndex;
- pointers[size++] = accumulatorIdx;
- }
- }
+ void add(int bufferIdx, int tIndex, int accumulatorIdx) {
+ while (size + 3 > pointers.length) {
+ pointers = Arrays.copyOf(pointers, pointers.length * 2);
+ }
+ pointers[size++] = bufferIdx;
+ pointers[size++] = tIndex;
+ pointers[size++] = accumulatorIdx;
+ }
+ }
- private static final int INIT_AGG_STATE_SIZE = 8;
- private final IHyracksTaskContext ctx;
+ private static final int INIT_AGG_STATE_SIZE = 8;
+ private final IHyracksTaskContext ctx;
- private final List<ByteBuffer> buffers;
- private final Link[] table;
- /**
- * Aggregate states: a list of states for all groups maintained in the main
- * memory.
- */
- private AggregateState[] aggregateStates;
- private int accumulatorSize;
+ private final List<ByteBuffer> buffers;
+ private final Link[] table;
+ /**
+ * Aggregate states: a list of states for all groups maintained in the main
+ * memory.
+ */
+ private AggregateState[] aggregateStates;
+ private int accumulatorSize;
- private int lastBIndex;
- private final int[] storedKeys;
- private final IBinaryComparator[] comparators;
- private final FrameTuplePairComparator ftpc;
- private final ITuplePartitionComputer tpc;
- private final IAggregatorDescriptor aggregator;
+ private int lastBIndex;
+ private final int[] storedKeys;
+ private final int[] keys;
+ private final IBinaryComparator[] comparators;
+ private final FrameTuplePairComparator ftpc;
+ private final ITuplePartitionComputer tpc;
+ private final IAggregatorDescriptor aggregator;
- private final FrameTupleAppender appender;
+ private final FrameTupleAppender appender;
- private final FrameTupleAccessor storedKeysAccessor;
+ private final FrameTupleAccessor storedKeysAccessor;
- private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
+ private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
- GroupingHashTable(IHyracksTaskContext ctx, int[] fields,
- IBinaryComparatorFactory[] comparatorFactories,
- ITuplePartitionComputerFactory tpcf,
- IAggregatorDescriptorFactory aggregatorFactory,
- RecordDescriptor inRecordDescriptor,
- RecordDescriptor outRecordDescriptor, int tableSize)
- throws HyracksDataException {
- this.ctx = ctx;
+ GroupingHashTable(IHyracksTaskContext ctx, int[] fields,
+ IBinaryComparatorFactory[] comparatorFactories,
+ ITuplePartitionComputerFactory tpcf,
+ IAggregatorDescriptorFactory aggregatorFactory,
+ RecordDescriptor inRecordDescriptor,
+ RecordDescriptor outRecordDescriptor, int tableSize)
+ throws HyracksDataException {
+ this.ctx = ctx;
- buffers = new ArrayList<ByteBuffer>();
- table = new Link[tableSize];
+ buffers = new ArrayList<ByteBuffer>();
+ table = new Link[tableSize];
- storedKeys = new int[fields.length];
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
- for (int i = 0; i < fields.length; ++i) {
- storedKeys[i] = i;
- storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
- }
+ keys = fields;
+ storedKeys = new int[fields.length];
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
+ for (int i = 0; i < fields.length; ++i) {
+ storedKeys[i] = i;
+ storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
+ }
- comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
- tpc = tpcf.createPartitioner();
+ comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
+ tpc = tpcf.createPartitioner();
- int[] keyFieldsInPartialResults = new int[fields.length];
- for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
- keyFieldsInPartialResults[i] = i;
- }
+ int[] keyFieldsInPartialResults = new int[fields.length];
+ for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+ keyFieldsInPartialResults[i] = i;
+ }
- this.aggregator = aggregatorFactory.createAggregator(ctx,
- inRecordDescriptor, outRecordDescriptor, fields,
- keyFieldsInPartialResults);
+ this.aggregator = aggregatorFactory.createAggregator(ctx,
+ inRecordDescriptor, outRecordDescriptor, fields,
+ keyFieldsInPartialResults);
- this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
- accumulatorSize = 0;
+ this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
+ accumulatorSize = 0;
- RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(
- storedKeySerDeser);
- storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
- storedKeysRecordDescriptor);
- lastBIndex = -1;
-
- appender = new FrameTupleAppender(ctx.getFrameSize());
-
- addNewBuffer();
+ RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(
+ storedKeySerDeser);
+ storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+ storedKeysRecordDescriptor);
+ lastBIndex = -1;
- if (fields.length < outRecordDescriptor.getFields().length) {
- stateTupleBuilder = new ArrayTupleBuilder(
- outRecordDescriptor.getFields().length);
- } else {
- stateTupleBuilder = new ArrayTupleBuilder(
- outRecordDescriptor.getFields().length + 1);
- }
- outputTupleBuilder = new ArrayTupleBuilder(
- outRecordDescriptor.getFields().length);
- }
+ appender = new FrameTupleAppender(ctx.getFrameSize());
- private void addNewBuffer() {
- ByteBuffer buffer = ctx.allocateFrame();
- buffer.position(0);
- buffer.limit(buffer.capacity());
- buffers.add(buffer);
- appender.reset(buffer, true);
- ++lastBIndex;
- }
+ addNewBuffer();
- void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
- int entry = tpc.partition(accessor, tIndex, table.length);
- Link link = table[entry];
- if (link == null) {
- link = table[entry] = new Link();
- }
- int saIndex = -1;
- for (int i = 0; i < link.size; i += 3) {
- int sbIndex = link.pointers[i];
- int stIndex = link.pointers[i + 1];
- storedKeysAccessor.reset(buffers.get(sbIndex));
- int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
- if (c == 0) {
- saIndex = link.pointers[i + 2];
- break;
- }
- }
- if (saIndex < 0) {
- // Did not find the key. Insert a new entry.
- saIndex = accumulatorSize++;
- // Add keys
+ if (fields.length < outRecordDescriptor.getFields().length) {
+ stateTupleBuilder = new ArrayTupleBuilder(
+ outRecordDescriptor.getFields().length);
+ } else {
+ stateTupleBuilder = new ArrayTupleBuilder(
+ outRecordDescriptor.getFields().length + 1);
+ }
+ outputTupleBuilder = new ArrayTupleBuilder(
+ outRecordDescriptor.getFields().length);
+ }
- // Add aggregation fields
- AggregateState newState = aggregator.createAggregateStates();
+ private void addNewBuffer() {
+ ByteBuffer buffer = ctx.allocateFrame();
+ buffer.position(0);
+ buffer.limit(buffer.capacity());
+ buffers.add(buffer);
+ appender.reset(buffer, true);
+ ++lastBIndex;
+ }
- stateTupleBuilder.reset();
+ void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
+ int entry = tpc.partition(accessor, tIndex, table.length);
+ Link link = table[entry];
+ if (link == null) {
+ link = table[entry] = new Link();
+ }
+ int saIndex = -1;
+ for (int i = 0; i < link.size; i += 3) {
+ int sbIndex = link.pointers[i];
+ int stIndex = link.pointers[i + 1];
+ storedKeysAccessor.reset(buffers.get(sbIndex));
+ int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
+ if (c == 0) {
+ saIndex = link.pointers[i + 2];
+ break;
+ }
+ }
+ if (saIndex < 0) {
+ // Did not find the key. Insert a new entry.
+ saIndex = accumulatorSize++;
+ // Add keys
- aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
+ // Add aggregation fields
+ AggregateState newState = aggregator.createAggregateStates();
- if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
- stateTupleBuilder.getByteArray(), 0,
- stateTupleBuilder.getSize())) {
- addNewBuffer();
- if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
- stateTupleBuilder.getByteArray(), 0,
- stateTupleBuilder.getSize())) {
- throw new HyracksDataException(
- "Cannot init the aggregate state in a single frame.");
- }
- }
-
- if (accumulatorSize >= aggregateStates.length) {
- aggregateStates = Arrays.copyOf(aggregateStates,
- aggregateStates.length * 2);
- }
+ stateTupleBuilder.reset();
+ for (int k = 0; k < keys.length; k++) {
+ stateTupleBuilder.addField(accessor, tIndex, keys[k]);
+ }
- aggregateStates[saIndex] = newState;
+ aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
- link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
+ if (!appender.appendSkipEmptyField(
+ stateTupleBuilder.getFieldEndOffsets(),
+ stateTupleBuilder.getByteArray(), 0,
+ stateTupleBuilder.getSize())) {
+ addNewBuffer();
+ if (!appender.appendSkipEmptyField(
+ stateTupleBuilder.getFieldEndOffsets(),
+ stateTupleBuilder.getByteArray(), 0,
+ stateTupleBuilder.getSize())) {
+ throw new HyracksDataException(
+ "Cannot init the aggregate state in a single frame.");
+ }
+ }
- } else {
- aggregator.aggregate(accessor, tIndex, null, 0,
- aggregateStates[saIndex]);
- }
- }
+ if (accumulatorSize >= aggregateStates.length) {
+ aggregateStates = Arrays.copyOf(aggregateStates,
+ aggregateStates.length * 2);
+ }
- void write(IFrameWriter writer) throws HyracksDataException {
- ByteBuffer buffer = ctx.allocateFrame();
- appender.reset(buffer, true);
+ aggregateStates[saIndex] = newState;
- for (int i = 0; i < table.length; ++i) {
- Link link = table[i];
- if (link != null) {
- for (int j = 0; j < link.size; j += 3) {
- int bIndex = link.pointers[j];
- int tIndex = link.pointers[j + 1];
- int aIndex = link.pointers[j + 2];
- ByteBuffer keyBuffer = buffers.get(bIndex);
- storedKeysAccessor.reset(keyBuffer);
-
- outputTupleBuilder.reset();
+ link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
- aggregator
- .outputFinalResult(outputTupleBuilder,
- storedKeysAccessor, tIndex,
- aggregateStates[aIndex]);
-
- if(!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(), outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())){
- writer.nextFrame(buffer);
- appender.reset(buffer, true);
- if(!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(), outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
- throw new HyracksDataException("Cannot write the aggregation output into a frame.");
- }
- }
+ } else {
+ aggregator.aggregate(accessor, tIndex, null, 0,
+ aggregateStates[saIndex]);
+ }
+ }
- }
- }
- }
- if (appender.getTupleCount() != 0) {
- writer.nextFrame(buffer);
- }
- }
+ void write(IFrameWriter writer) throws HyracksDataException {
+ ByteBuffer buffer = ctx.allocateFrame();
+ appender.reset(buffer, true);
- void close() throws HyracksDataException {
- for (AggregateState aState : aggregateStates) {
- aState.close();
- }
- }
+ for (int i = 0; i < table.length; ++i) {
+ Link link = table[i];
+ if (link != null) {
+ for (int j = 0; j < link.size; j += 3) {
+ int bIndex = link.pointers[j];
+ int tIndex = link.pointers[j + 1];
+ int aIndex = link.pointers[j + 2];
+ ByteBuffer keyBuffer = buffers.get(bIndex);
+ storedKeysAccessor.reset(keyBuffer);
+
+ // copy keys
+ outputTupleBuilder.reset();
+ for (int k = 0; k < storedKeys.length; k++) {
+ outputTupleBuilder.addField(storedKeysAccessor, tIndex,
+ storedKeys[k]);
+ }
+
+ aggregator
+ .outputFinalResult(outputTupleBuilder,
+ storedKeysAccessor, tIndex,
+ aggregateStates[aIndex]);
+
+ if (!appender.appendSkipEmptyField(
+ outputTupleBuilder.getFieldEndOffsets(),
+ outputTupleBuilder.getByteArray(), 0,
+ outputTupleBuilder.getSize())) {
+ writer.nextFrame(buffer);
+ appender.reset(buffer, true);
+ if (!appender.appendSkipEmptyField(
+ outputTupleBuilder.getFieldEndOffsets(),
+ outputTupleBuilder.getByteArray(), 0,
+ outputTupleBuilder.getSize())) {
+ throw new HyracksDataException(
+ "Cannot write the aggregation output into a frame.");
+ }
+ }
+
+ }
+ }
+ }
+ if (appender.getTupleCount() != 0) {
+ writer.nextFrame(buffer);
+ }
+ }
+
+ void close() throws HyracksDataException {
+ for (AggregateState aState : aggregateStates) {
+ aState.close();
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 6bd053f..29bd083 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -44,74 +44,74 @@
*/
public class HashSpillableTableFactory implements ISpillableTableFactory {
- private static final long serialVersionUID = 1L;
- private final ITuplePartitionComputerFactory tpcf;
- private final int tableSize;
+ private static final long serialVersionUID = 1L;
+ private final ITuplePartitionComputerFactory tpcf;
+ private final int tableSize;
- public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf,
- int tableSize) {
- this.tpcf = tpcf;
- this.tableSize = tableSize;
- }
+ public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf,
+ int tableSize) {
+ this.tpcf = tpcf;
+ this.tableSize = tableSize;
+ }
- /*
- * (non-Javadoc)
- *
- * @see
- * edu.uci.ics.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
- * buildSpillableTable(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
- * int[], edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
- * edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
- * edu.
- * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
- * [], edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
- * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int)
- */
- @Override
- public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx,
- final int[] keyFields,
- IBinaryComparatorFactory[] comparatorFactories,
- INormalizedKeyComputerFactory firstKeyNormalizerFactory,
- IAggregatorDescriptorFactory aggregateFactory,
- RecordDescriptor inRecordDescriptor,
- RecordDescriptor outRecordDescriptor, final int framesLimit)
- throws HyracksDataException {
- final int[] storedKeys = new int[keyFields.length];
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[keyFields.length];
- for (int i = 0; i < keyFields.length; i++) {
- storedKeys[i] = i;
- storedKeySerDeser[i] = inRecordDescriptor.getFields()[keyFields[i]];
- }
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * edu.uci.ics.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
+ * buildSpillableTable(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+ * int[], edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
+ * edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
+ * edu.
+ * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
+ * [], edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+ * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int)
+ */
+ @Override
+ public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx,
+ final int[] keyFields,
+ IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+ IAggregatorDescriptorFactory aggregateFactory,
+ RecordDescriptor inRecordDescriptor,
+ RecordDescriptor outRecordDescriptor, final int framesLimit)
+ throws HyracksDataException {
+ final int[] storedKeys = new int[keyFields.length];
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[keyFields.length];
+ for (int i = 0; i < keyFields.length; i++) {
+ storedKeys[i] = i;
+ storedKeySerDeser[i] = inRecordDescriptor.getFields()[keyFields[i]];
+ }
- RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
- final FrameTupleAccessor storedKeysAccessor1;
- final FrameTupleAccessor storedKeysAccessor2;
- if (keyFields.length >= outRecordDescriptor.getFields().length) {
- // for the case of zero-aggregations
- ISerializerDeserializer<?>[] fields = outRecordDescriptor
- .getFields();
- ITypeTrait[] types = outRecordDescriptor.getTypeTraits();
- ISerializerDeserializer<?>[] newFields = new ISerializerDeserializer[fields.length + 1];
- for (int i = 0; i < fields.length; i++)
- newFields[i] = fields[i];
- ITypeTrait[] newTypes = null;
- if (types != null) {
- newTypes = new ITypeTrait[types.length + 1];
- for (int i = 0; i < types.length; i++)
- newTypes[i] = types[i];
- }
- internalRecordDescriptor = new RecordDescriptor(newFields, newTypes);
- }
- storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
- internalRecordDescriptor);
- storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
- internalRecordDescriptor);
+ RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
+ final FrameTupleAccessor storedKeysAccessor1;
+ final FrameTupleAccessor storedKeysAccessor2;
+ if (keyFields.length >= outRecordDescriptor.getFields().length) {
+ // for the case of zero-aggregations
+ ISerializerDeserializer<?>[] fields = outRecordDescriptor
+ .getFields();
+ ITypeTraits[] types = outRecordDescriptor.getTypeTraits();
+ ISerializerDeserializer<?>[] newFields = new ISerializerDeserializer[fields.length + 1];
+ for (int i = 0; i < fields.length; i++)
+ newFields[i] = fields[i];
+ ITypeTraits[] newTypes = null;
+ if (types != null) {
+ newTypes = new ITypeTraits[types.length + 1];
+ for (int i = 0; i < types.length; i++)
+ newTypes[i] = types[i];
+ }
+ internalRecordDescriptor = new RecordDescriptor(newFields, newTypes);
+ }
+ storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
+ internalRecordDescriptor);
+ storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
+ internalRecordDescriptor);
- final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
+ final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
final FrameTuplePairComparator ftpcPartial = new FrameTuplePairComparator(
keyFields, storedKeys, comparators);
@@ -119,22 +119,22 @@
final FrameTuplePairComparator ftpcTuple = new FrameTuplePairComparator(
storedKeys, storedKeys, comparators);
- final ITuplePartitionComputer tpc = tpcf.createPartitioner();
+ final ITuplePartitionComputer tpc = tpcf.createPartitioner();
- final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null
- : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+ final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null
+ : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- int[] keyFieldsInPartialResults = new int[keyFields.length];
- for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
- keyFieldsInPartialResults[i] = i;
- }
+ int[] keyFieldsInPartialResults = new int[keyFields.length];
+ for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+ keyFieldsInPartialResults[i] = i;
+ }
- final IAggregatorDescriptor aggregator = aggregateFactory
- .createAggregator(ctx, inRecordDescriptor, outRecordDescriptor,
- keyFields, keyFieldsInPartialResults);
+ final IAggregatorDescriptor aggregator = aggregateFactory
+ .createAggregator(ctx, inRecordDescriptor, outRecordDescriptor,
+ keyFields, keyFieldsInPartialResults);
- final AggregateState aggregateState = aggregator
- .createAggregateStates();
+ final AggregateState aggregateState = aggregator
+ .createAggregateStates();
final ArrayTupleBuilder stateTupleBuilder;
if (keyFields.length < outRecordDescriptor.getFields().length) {
@@ -152,7 +152,7 @@
private int lastBufIndex;
- private ByteBuffer outputFrame;
+ private ByteBuffer outputFrame;
private FrameTupleAppender outputAppender;
private FrameTupleAppender stateAppender = new FrameTupleAppender(
@@ -163,88 +163,88 @@
private final TuplePointer storedTuplePointer = new TuplePointer();
private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
- /**
- * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
- * = Frame index in the "Frames" list, [1] = Tuple index in the
- * frame, [2] = Poor man's normalized key for the tuple.
- */
- private int[] tPointers;
+ /**
+ * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
+ * = Frame index in the "Frames" list, [1] = Tuple index in the
+ * frame, [2] = Poor man's normalized key for the tuple.
+ */
+ private int[] tPointers;
- @Override
- public void sortFrames() {
- int sfIdx = storedKeys[0];
- int totalTCount = table.getTupleCount();
- tPointers = new int[totalTCount * 3];
- int ptr = 0;
+ @Override
+ public void sortFrames() {
+ int sfIdx = storedKeys[0];
+ int totalTCount = table.getTupleCount();
+ tPointers = new int[totalTCount * 3];
+ int ptr = 0;
- for (int i = 0; i < tableSize; i++) {
- int entry = i;
- int offset = 0;
- do {
- table.getTuplePointer(entry, offset, storedTuplePointer);
- if (storedTuplePointer.frameIndex < 0)
- break;
- tPointers[ptr * 3] = entry;
- tPointers[ptr * 3 + 1] = offset;
- table.getTuplePointer(entry, offset, storedTuplePointer);
- int fIndex = storedTuplePointer.frameIndex;
- int tIndex = storedTuplePointer.tupleIndex;
- storedKeysAccessor1.reset(frames.get(fIndex));
- int tStart = storedKeysAccessor1
- .getTupleStartOffset(tIndex);
- int f0StartRel = storedKeysAccessor1
- .getFieldStartOffset(tIndex, sfIdx);
- int f0EndRel = storedKeysAccessor1.getFieldEndOffset(
- tIndex, sfIdx);
- int f0Start = f0StartRel + tStart
- + storedKeysAccessor1.getFieldSlotsLength();
- tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc
- .normalize(storedKeysAccessor1.getBuffer()
- .array(), f0Start, f0EndRel
- - f0StartRel);
- ptr++;
- offset++;
- } while (true);
- }
- /**
- * Sort using quick sort
- */
- if (tPointers.length > 0) {
- sort(tPointers, 0, totalTCount);
- }
- }
+ for (int i = 0; i < tableSize; i++) {
+ int entry = i;
+ int offset = 0;
+ do {
+ table.getTuplePointer(entry, offset, storedTuplePointer);
+ if (storedTuplePointer.frameIndex < 0)
+ break;
+ tPointers[ptr * 3] = entry;
+ tPointers[ptr * 3 + 1] = offset;
+ table.getTuplePointer(entry, offset, storedTuplePointer);
+ int fIndex = storedTuplePointer.frameIndex;
+ int tIndex = storedTuplePointer.tupleIndex;
+ storedKeysAccessor1.reset(frames.get(fIndex));
+ int tStart = storedKeysAccessor1
+ .getTupleStartOffset(tIndex);
+ int f0StartRel = storedKeysAccessor1
+ .getFieldStartOffset(tIndex, sfIdx);
+ int f0EndRel = storedKeysAccessor1.getFieldEndOffset(
+ tIndex, sfIdx);
+ int f0Start = f0StartRel + tStart
+ + storedKeysAccessor1.getFieldSlotsLength();
+ tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc
+ .normalize(storedKeysAccessor1.getBuffer()
+ .array(), f0Start, f0EndRel
+ - f0StartRel);
+ ptr++;
+ offset++;
+ } while (true);
+ }
+ /**
+ * Sort using quick sort
+ */
+ if (tPointers.length > 0) {
+ sort(tPointers, 0, totalTCount);
+ }
+ }
- @Override
- public void reset() {
- lastBufIndex = -1;
- tPointers = null;
- table.reset();
- aggregator.reset();
- }
+ @Override
+ public void reset() {
+ lastBufIndex = -1;
+ tPointers = null;
+ table.reset();
+ aggregator.reset();
+ }
- @Override
- public boolean insert(FrameTupleAccessor accessor, int tIndex)
- throws HyracksDataException {
- if (lastBufIndex < 0)
- nextAvailableFrame();
- int entry = tpc.partition(accessor, tIndex, tableSize);
- boolean foundGroup = false;
- int offset = 0;
- do {
- table.getTuplePointer(entry, offset++, storedTuplePointer);
- if (storedTuplePointer.frameIndex < 0)
- break;
- storedKeysAccessor1.reset(frames
- .get(storedTuplePointer.frameIndex));
- int c = ftpcPartial.compare(accessor, tIndex,
- storedKeysAccessor1, storedTuplePointer.tupleIndex);
- if (c == 0) {
- foundGroup = true;
- break;
- }
- } while (true);
+ @Override
+ public boolean insert(FrameTupleAccessor accessor, int tIndex)
+ throws HyracksDataException {
+ if (lastBufIndex < 0)
+ nextAvailableFrame();
+ int entry = tpc.partition(accessor, tIndex, tableSize);
+ boolean foundGroup = false;
+ int offset = 0;
+ do {
+ table.getTuplePointer(entry, offset++, storedTuplePointer);
+ if (storedTuplePointer.frameIndex < 0)
+ break;
+ storedKeysAccessor1.reset(frames
+ .get(storedTuplePointer.frameIndex));
+ int c = ftpcPartial.compare(accessor, tIndex,
+ storedKeysAccessor1, storedTuplePointer.tupleIndex);
+ if (c == 0) {
+ foundGroup = true;
+ break;
+ }
+ } while (true);
- if (!foundGroup) {
+ if (!foundGroup) {
stateTupleBuilder.reset();
@@ -272,29 +272,29 @@
table.insert(entry, storedTuplePointer);
} else {
- aggregator.aggregate(accessor, tIndex, storedKeysAccessor1,
- storedTuplePointer.tupleIndex, aggregateState);
+ aggregator.aggregate(accessor, tIndex, storedKeysAccessor1,
+ storedTuplePointer.tupleIndex, aggregateState);
- }
- return true;
- }
+ }
+ return true;
+ }
- @Override
- public List<ByteBuffer> getFrames() {
- return frames;
- }
+ @Override
+ public List<ByteBuffer> getFrames() {
+ return frames;
+ }
- @Override
- public int getFrameCount() {
- return lastBufIndex;
- }
+ @Override
+ public int getFrameCount() {
+ return lastBufIndex;
+ }
- @Override
- public void flushFrames(IFrameWriter writer, boolean isPartial)
- throws HyracksDataException {
- if (outputFrame == null) {
- outputFrame = ctx.allocateFrame();
- }
+ @Override
+ public void flushFrames(IFrameWriter writer, boolean isPartial)
+ throws HyracksDataException {
+ if (outputFrame == null) {
+ outputFrame = ctx.allocateFrame();
+ }
if (outputAppender == null) {
outputAppender = new FrameTupleAppender(
@@ -305,24 +305,24 @@
writer.open();
- if (tPointers == null) {
- // Not sorted
- for (int i = 0; i < tableSize; ++i) {
- int entry = i;
- int offset = 0;
- do {
- table.getTuplePointer(entry, offset++,
- storedTuplePointer);
- if (storedTuplePointer.frameIndex < 0)
- break;
- int bIndex = storedTuplePointer.frameIndex;
- int tIndex = storedTuplePointer.tupleIndex;
+ if (tPointers == null) {
+ // Not sorted
+ for (int i = 0; i < tableSize; ++i) {
+ int entry = i;
+ int offset = 0;
+ do {
+ table.getTuplePointer(entry, offset++,
+ storedTuplePointer);
+ if (storedTuplePointer.frameIndex < 0)
+ break;
+ int bIndex = storedTuplePointer.frameIndex;
+ int tIndex = storedTuplePointer.tupleIndex;
- storedKeysAccessor1.reset(frames.get(bIndex));
+ storedKeysAccessor1.reset(frames.get(bIndex));
outputTupleBuilder.reset();
- if (isPartial) {
+ if (isPartial) {
aggregator.outputPartialResult(
outputTupleBuilder,
@@ -378,7 +378,7 @@
outputTupleBuilder.reset();
- if (isPartial) {
+ if (isPartial) {
aggregator
.outputPartialResult(outputTupleBuilder,
@@ -415,27 +415,27 @@
aggregator.close();
}
- @Override
- public void close() {
- lastBufIndex = -1;
- tPointers = null;
- table.close();
- frames.clear();
- aggregateState.close();
- }
+ @Override
+ public void close() {
+ lastBufIndex = -1;
+ tPointers = null;
+ table.close();
+ frames.clear();
+ aggregateState.close();
+ }
- /**
- * Set the working frame to the next available frame in the frame
- * list. There are two cases:<br>
- * 1) If the next frame is not initialized, allocate a new frame. 2)
- * When frames are already created, they are recycled.
- *
- * @return Whether a new frame is added successfully.
- */
- private boolean nextAvailableFrame() {
- // Return false if the number of frames is equal to the limit.
- if (lastBufIndex + 1 >= framesLimit)
- return false;
+ /**
+ * Set the working frame to the next available frame in the frame
+ * list. There are two cases:<br>
+ * 1) If the next frame is not initialized, allocate a new frame. 2)
+ * When frames are already created, they are recycled.
+ *
+ * @return Whether a new frame is added successfully.
+ */
+ private boolean nextAvailableFrame() {
+ // Return false if the number of frames is equal to the limit.
+ if (lastBufIndex + 1 >= framesLimit)
+ return false;
if (frames.size() < framesLimit) {
// Insert a new frame
@@ -456,107 +456,107 @@
return true;
}
- private void sort(int[] tPointers, int offset, int length) {
- int m = offset + (length >> 1);
- int mTable = tPointers[m * 3];
- int mRow = tPointers[m * 3 + 1];
- int mNormKey = tPointers[m * 3 + 2];
+ private void sort(int[] tPointers, int offset, int length) {
+ int m = offset + (length >> 1);
+ int mTable = tPointers[m * 3];
+ int mRow = tPointers[m * 3 + 1];
+ int mNormKey = tPointers[m * 3 + 2];
- table.getTuplePointer(mTable, mRow, storedTuplePointer);
- int mFrame = storedTuplePointer.frameIndex;
- int mTuple = storedTuplePointer.tupleIndex;
- storedKeysAccessor1.reset(frames.get(mFrame));
+ table.getTuplePointer(mTable, mRow, storedTuplePointer);
+ int mFrame = storedTuplePointer.frameIndex;
+ int mTuple = storedTuplePointer.tupleIndex;
+ storedKeysAccessor1.reset(frames.get(mFrame));
- int a = offset;
- int b = a;
- int c = offset + length - 1;
- int d = c;
- while (true) {
- while (b <= c) {
- int bTable = tPointers[b * 3];
- int bRow = tPointers[b * 3 + 1];
- int bNormKey = tPointers[b * 3 + 2];
- int cmp = 0;
- if (bNormKey != mNormKey) {
- cmp = ((((long) bNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
- : 1;
- } else {
- table.getTuplePointer(bTable, bRow,
- storedTuplePointer);
- int bFrame = storedTuplePointer.frameIndex;
- int bTuple = storedTuplePointer.tupleIndex;
- storedKeysAccessor2.reset(frames.get(bFrame));
- cmp = ftpcTuple.compare(storedKeysAccessor2,
- bTuple, storedKeysAccessor1, mTuple);
- }
- if (cmp > 0) {
- break;
- }
- if (cmp == 0) {
- swap(tPointers, a++, b);
- }
- ++b;
- }
- while (c >= b) {
- int cTable = tPointers[c * 3];
- int cRow = tPointers[c * 3 + 1];
- int cNormKey = tPointers[c * 3 + 2];
- int cmp = 0;
- if (cNormKey != mNormKey) {
- cmp = ((((long) cNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
- : 1;
- } else {
- table.getTuplePointer(cTable, cRow,
- storedTuplePointer);
- int cFrame = storedTuplePointer.frameIndex;
- int cTuple = storedTuplePointer.tupleIndex;
- storedKeysAccessor2.reset(frames.get(cFrame));
- cmp = ftpcTuple.compare(storedKeysAccessor2,
- cTuple, storedKeysAccessor1, mTuple);
- }
- if (cmp < 0) {
- break;
- }
- if (cmp == 0) {
- swap(tPointers, c, d--);
- }
- --c;
- }
- if (b > c)
- break;
- swap(tPointers, b++, c--);
- }
+ int a = offset;
+ int b = a;
+ int c = offset + length - 1;
+ int d = c;
+ while (true) {
+ while (b <= c) {
+ int bTable = tPointers[b * 3];
+ int bRow = tPointers[b * 3 + 1];
+ int bNormKey = tPointers[b * 3 + 2];
+ int cmp = 0;
+ if (bNormKey != mNormKey) {
+ cmp = ((((long) bNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
+ : 1;
+ } else {
+ table.getTuplePointer(bTable, bRow,
+ storedTuplePointer);
+ int bFrame = storedTuplePointer.frameIndex;
+ int bTuple = storedTuplePointer.tupleIndex;
+ storedKeysAccessor2.reset(frames.get(bFrame));
+ cmp = ftpcTuple.compare(storedKeysAccessor2,
+ bTuple, storedKeysAccessor1, mTuple);
+ }
+ if (cmp > 0) {
+ break;
+ }
+ if (cmp == 0) {
+ swap(tPointers, a++, b);
+ }
+ ++b;
+ }
+ while (c >= b) {
+ int cTable = tPointers[c * 3];
+ int cRow = tPointers[c * 3 + 1];
+ int cNormKey = tPointers[c * 3 + 2];
+ int cmp = 0;
+ if (cNormKey != mNormKey) {
+ cmp = ((((long) cNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
+ : 1;
+ } else {
+ table.getTuplePointer(cTable, cRow,
+ storedTuplePointer);
+ int cFrame = storedTuplePointer.frameIndex;
+ int cTuple = storedTuplePointer.tupleIndex;
+ storedKeysAccessor2.reset(frames.get(cFrame));
+ cmp = ftpcTuple.compare(storedKeysAccessor2,
+ cTuple, storedKeysAccessor1, mTuple);
+ }
+ if (cmp < 0) {
+ break;
+ }
+ if (cmp == 0) {
+ swap(tPointers, c, d--);
+ }
+ --c;
+ }
+ if (b > c)
+ break;
+ swap(tPointers, b++, c--);
+ }
- int s;
- int n = offset + length;
- s = Math.min(a - offset, b - a);
- vecswap(tPointers, offset, b - s, s);
- s = Math.min(d - c, n - d - 1);
- vecswap(tPointers, b, n - s, s);
+ int s;
+ int n = offset + length;
+ s = Math.min(a - offset, b - a);
+ vecswap(tPointers, offset, b - s, s);
+ s = Math.min(d - c, n - d - 1);
+ vecswap(tPointers, b, n - s, s);
- if ((s = b - a) > 1) {
- sort(tPointers, offset, s);
- }
- if ((s = d - c) > 1) {
- sort(tPointers, n - s, s);
- }
- }
+ if ((s = b - a) > 1) {
+ sort(tPointers, offset, s);
+ }
+ if ((s = d - c) > 1) {
+ sort(tPointers, n - s, s);
+ }
+ }
- private void swap(int x[], int a, int b) {
- for (int i = 0; i < 3; ++i) {
- int t = x[a * 3 + i];
- x[a * 3 + i] = x[b * 3 + i];
- x[b * 3 + i] = t;
- }
- }
+ private void swap(int x[], int a, int b) {
+ for (int i = 0; i < 3; ++i) {
+ int t = x[a * 3 + i];
+ x[a * 3 + i] = x[b * 3 + i];
+ x[b * 3 + i] = t;
+ }
+ }
- private void vecswap(int x[], int a, int b, int n) {
- for (int i = 0; i < n; i++, a++, b++) {
- swap(x, a, b);
- }
- }
+ private void vecswap(int x[], int a, int b, int n) {
+ for (int i = 0; i < n; i++, a++, b++) {
+ swap(x, a, b);
+ }
+ }
- };
- }
+ };
+ }
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
index 226642f..ee8489a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
@@ -27,145 +27,157 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class PreclusteredGroupWriter implements IFrameWriter {
- private final int[] groupFields;
- private final IBinaryComparator[] comparators;
- private final IAggregatorDescriptor aggregator;
- private final AggregateState aggregateState;
- private final IFrameWriter writer;
- private final ByteBuffer copyFrame;
- private final FrameTupleAccessor inFrameAccessor;
- private final FrameTupleAccessor copyFrameAccessor;
+ private final int[] groupFields;
+ private final IBinaryComparator[] comparators;
+ private final IAggregatorDescriptor aggregator;
+ private final AggregateState aggregateState;
+ private final IFrameWriter writer;
+ private final ByteBuffer copyFrame;
+ private final FrameTupleAccessor inFrameAccessor;
+ private final FrameTupleAccessor copyFrameAccessor;
- private final ByteBuffer outFrame;
- private final FrameTupleAppender appender;
- private final ArrayTupleBuilder tupleBuilder;
+ private final ByteBuffer outFrame;
+ private final FrameTupleAppender appender;
+ private final ArrayTupleBuilder tupleBuilder;
- private boolean first;
+ private boolean first;
- public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields,
- IBinaryComparator[] comparators, IAggregatorDescriptor aggregator,
- RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, IFrameWriter writer) {
- this.groupFields = groupFields;
- this.comparators = comparators;
- this.aggregator = aggregator;
- this.aggregateState = aggregator.createAggregateStates();
- this.writer = writer;
- copyFrame = ctx.allocateFrame();
- inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
- inRecordDesc);
- copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
- inRecordDesc);
- copyFrameAccessor.reset(copyFrame);
+ public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields,
+ IBinaryComparator[] comparators, IAggregatorDescriptor aggregator,
+ RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
+ IFrameWriter writer) {
+ this.groupFields = groupFields;
+ this.comparators = comparators;
+ this.aggregator = aggregator;
+ this.aggregateState = aggregator.createAggregateStates();
+ this.writer = writer;
+ copyFrame = ctx.allocateFrame();
+ inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+ inRecordDesc);
+ copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+ inRecordDesc);
+ copyFrameAccessor.reset(copyFrame);
- outFrame = ctx.allocateFrame();
- appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(outFrame, true);
-
- tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
- }
+ outFrame = ctx.allocateFrame();
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(outFrame, true);
- @Override
- public void open() throws HyracksDataException {
- writer.open();
- first = true;
- }
+ tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
+ }
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- inFrameAccessor.reset(buffer);
- int nTuples = inFrameAccessor.getTupleCount();
- for (int i = 0; i < nTuples; ++i) {
- if (first) {
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ first = true;
+ }
- tupleBuilder.reset();
-
- aggregator.init(tupleBuilder, inFrameAccessor, i, aggregateState);
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ inFrameAccessor.reset(buffer);
+ int nTuples = inFrameAccessor.getTupleCount();
+ for (int i = 0; i < nTuples; ++i) {
+ if (first) {
- first = false;
+ tupleBuilder.reset();
+ for (int j = 0; j < groupFields.length; j++) {
+ tupleBuilder.addField(inFrameAccessor, i, j);
+ }
+ aggregator.init(tupleBuilder, inFrameAccessor, i,
+ aggregateState);
- } else {
- if (i == 0) {
- switchGroupIfRequired(copyFrameAccessor,
- copyFrameAccessor.getTupleCount() - 1,
- inFrameAccessor, i);
- } else {
- switchGroupIfRequired(inFrameAccessor, i - 1,
- inFrameAccessor, i);
- }
+ first = false;
- }
- }
- FrameUtils.copy(buffer, copyFrame);
- }
+ } else {
+ if (i == 0) {
+ switchGroupIfRequired(copyFrameAccessor,
+ copyFrameAccessor.getTupleCount() - 1,
+ inFrameAccessor, i);
+ } else {
+ switchGroupIfRequired(inFrameAccessor, i - 1,
+ inFrameAccessor, i);
+ }
- private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor,
- int prevTupleIndex, FrameTupleAccessor currTupleAccessor,
- int currTupleIndex) throws HyracksDataException {
- if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor,
- currTupleIndex)) {
- writeOutput(prevTupleAccessor, prevTupleIndex);
+ }
+ }
+ FrameUtils.copy(buffer, copyFrame);
+ }
- tupleBuilder.reset();
- aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex,
- aggregateState);
- } else {
- aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0,
- aggregateState);
- }
- }
+ private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor,
+ int prevTupleIndex, FrameTupleAccessor currTupleAccessor,
+ int currTupleIndex) throws HyracksDataException {
+ if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor,
+ currTupleIndex)) {
+ writeOutput(prevTupleAccessor, prevTupleIndex);
- private void writeOutput(final FrameTupleAccessor lastTupleAccessor,
- int lastTupleIndex) throws HyracksDataException {
+ tupleBuilder.reset();
+ for (int j = 0; j < groupFields.length; j++) {
+ tupleBuilder.addField(currTupleAccessor, currTupleIndex, j);
+ }
+ aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex,
+ aggregateState);
+ } else {
+ aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0,
+ aggregateState);
+ }
+ }
- tupleBuilder.reset();
+ private void writeOutput(final FrameTupleAccessor lastTupleAccessor,
+ int lastTupleIndex) throws HyracksDataException {
- aggregator.outputFinalResult(tupleBuilder,
- lastTupleAccessor, lastTupleIndex, aggregateState);
-
- if(!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
- FrameUtils.flushFrame(outFrame, writer);
- appender.reset(outFrame, true);
- if(!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
- throw new HyracksDataException(
- "The output cannot be fit into a frame.");
- }
- }
+ tupleBuilder.reset();
+ for (int j = 0; j < groupFields.length; j++) {
+ tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, j);
+ }
+ aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor,
+ lastTupleIndex, aggregateState);
- }
+ if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
+ tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+ FrameUtils.flushFrame(outFrame, writer);
+ appender.reset(outFrame, true);
+ if (!appender.appendSkipEmptyField(
+ tupleBuilder.getFieldEndOffsets(),
+ tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+ throw new HyracksDataException(
+ "The output cannot be fit into a frame.");
+ }
+ }
- private boolean sameGroup(FrameTupleAccessor a1, int t1Idx,
- FrameTupleAccessor a2, int t2Idx) {
- for (int i = 0; i < comparators.length; ++i) {
- int fIdx = groupFields[i];
- int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength()
- + a1.getFieldStartOffset(t1Idx, fIdx);
- int l1 = a1.getFieldLength(t1Idx, fIdx);
- int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength()
- + a2.getFieldStartOffset(t2Idx, fIdx);
- int l2 = a2.getFieldLength(t2Idx, fIdx);
- if (comparators[i].compare(a1.getBuffer().array(), s1, l1, a2
- .getBuffer().array(), s2, l2) != 0) {
- return false;
- }
- }
- return true;
- }
+ }
- @Override
- public void fail() throws HyracksDataException {
- writer.fail();
- }
+ private boolean sameGroup(FrameTupleAccessor a1, int t1Idx,
+ FrameTupleAccessor a2, int t2Idx) {
+ for (int i = 0; i < comparators.length; ++i) {
+ int fIdx = groupFields[i];
+ int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength()
+ + a1.getFieldStartOffset(t1Idx, fIdx);
+ int l1 = a1.getFieldLength(t1Idx, fIdx);
+ int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength()
+ + a2.getFieldStartOffset(t2Idx, fIdx);
+ int l2 = a2.getFieldLength(t2Idx, fIdx);
+ if (comparators[i].compare(a1.getBuffer().array(), s1, l1, a2
+ .getBuffer().array(), s2, l2) != 0) {
+ return false;
+ }
+ }
+ return true;
+ }
- @Override
- public void close() throws HyracksDataException {
- if (!first) {
- writeOutput(copyFrameAccessor,
- copyFrameAccessor.getTupleCount() - 1);
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outFrame, writer);
- }
- }
- aggregateState.close();
- writer.close();
- }
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (!first) {
+ writeOutput(copyFrameAccessor,
+ copyFrameAccessor.getTupleCount() - 1);
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(outFrame, writer);
+ }
+ }
+ aggregateState.close();
+ writer.close();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks-examples/btree-example/btreeclient/pom.xml
index a95d17e..e0930b2 100644
--- a/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -2,8 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
<artifactId>btreeclient</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 4711e9e..5a6d8cd 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -18,21 +18,21 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -63,7 +63,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -86,7 +86,7 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job = createJob(options);
@@ -134,16 +134,16 @@
// prepare insertion into primary index
// tuples to be put into B-Tree shall have 4 fields
int primaryFieldCount = 4;
- ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- primaryTypeTraits[0] = new TypeTrait(4);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(4);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+ primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
// comparator factories for primary index
IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[1];
- primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
// create factories and providers for secondary B-Tree
TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(primaryTupleWriterFactory);
@@ -168,15 +168,15 @@
// prepare insertion into secondary index
// tuples to be put into B-Tree shall have 2 fields
int secondaryFieldCount = 2;
- ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
- secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryTypeTraits[1] = new TypeTrait(4);
+ ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+ secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
// comparator factories for secondary index
IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
- secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+ secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
// create factories and providers for secondary B-Tree
TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
@@ -202,7 +202,7 @@
// distribute the records from the datagen via hashing to the bulk load
// ops
IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
- hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
+ hashFactories[0] = PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY);
IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 9790496..0b1cd6c 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -17,20 +17,21 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -60,7 +61,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -83,7 +84,7 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job = createJob(options);
@@ -130,18 +131,18 @@
int[] sortFields = { 2 };
// comparators for sort fields
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+ comparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, options.sbSize, sortFields,
comparatorFactories, recDesc);
JobHelper.createPartitionConstraint(spec, sorter, splitNCs);
// tuples to be put into B-Tree shall have 4 fields
int fieldCount = 4;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- typeTraits[2] = new TypeTrait(4);
- typeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
// create factories and providers for B-Tree
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
@@ -165,7 +166,7 @@
// distribute the records from the datagen via hashing to the bulk load
// ops
IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
- hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
+ hashFactories[0] = PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY);
IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 4929243..aa73a62 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -19,18 +19,19 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -59,7 +60,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -76,7 +77,7 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job = createJob(options);
@@ -95,16 +96,16 @@
String[] splitNCs = options.ncs.split(",");
int fieldCount = 4;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- typeTraits[2] = new TypeTrait(4);
- typeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
// comparators for btree
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ comparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
// create factories and providers for B-Tree
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index d77eb01..ad9ad3a 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -17,17 +17,17 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -56,7 +56,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -79,7 +79,7 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job = createJob(options);
@@ -111,17 +111,17 @@
IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int primaryFieldCount = 4;
- ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- primaryTypeTraits[0] = new TypeTrait(4);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(4);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+ primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
// comparators for sort fields and BTree fields
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[2];
- comparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- comparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ comparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+ comparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
// create factories and providers for primary B-Tree
TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(primaryTupleWriterFactory);
@@ -144,9 +144,9 @@
// tuples to be put into B-Tree shall have 2 fields
int secondaryFieldCount = 2;
- ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
- secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryTypeTraits[1] = new TypeTrait(4);
+ ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+ secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
// create factories and providers for secondary B-Tree
TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 3a79df6..2719397 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -19,19 +19,19 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -60,7 +60,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -80,7 +80,7 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job = createJob(options);
@@ -106,19 +106,19 @@
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int secondaryFieldCount = 2;
- ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
- secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryTypeTraits[1] = new TypeTrait(4);
+ ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+ secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
// comparators for sort fields and BTree fields
IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
- secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+ secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
// comparators for primary index
IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[1];
- primaryComparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ primaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
// create factories and providers for secondary B-Tree
TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
@@ -131,11 +131,11 @@
IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE, });
int primaryFieldCount = 4;
- ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- primaryTypeTraits[0] = new TypeTrait(4);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(4);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+ primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
// create factories and providers for secondary B-Tree
TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -147,13 +147,13 @@
// i.e. we will have a range condition on the first field only (implying
// [-infinity, +infinity] for the second field)
IBinaryComparatorFactory[] searchComparatorFactories = new IBinaryComparatorFactory[1];
- searchComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-
+ searchComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
// build tuple containing low and high search keys
ArrayTupleBuilder tb = new ArrayTupleBuilder(searchComparatorFactories.length * 2); // low
- // and
- // high
- // key
+ // and
+ // high
+ // key
DataOutput dos = tb.getDataOutput();
tb.reset();
diff --git a/hyracks-examples/btree-example/btreehelper/pom.xml b/hyracks-examples/btree-example/btreehelper/pom.xml
index 2615dc2..046d762 100644
--- a/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -2,8 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
<artifactId>btreehelper</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
@@ -29,6 +27,11 @@
<version>0.2.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index 35deb57..36e315d 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -58,8 +58,8 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final ByteBuffer outputFrame = ctx.allocateFrame();
final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -82,7 +82,7 @@
appender.reset(outputFrame, true);
for (int i = 0; i < numRecords; i++) {
tb.reset();
- for (int j = 0; j < recDesc.getFields().length; j++) {
+ for (int j = 0; j < recDesc.getFieldCount(); j++) {
genField(tb, j);
}
diff --git a/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-examples/hyracks-integration-tests/pom.xml
index 7012cde..d74b384 100644
--- a/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -2,8 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>hyracks-integration-tests</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
@@ -80,5 +78,10 @@
<type>jar</type>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 38dd051..6ef1740 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -28,12 +28,13 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -76,7 +77,7 @@
// field, type and key declarations for primary index
private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private int primaryKeyFieldCount = 1;
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -98,13 +99,13 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
- primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
loadPrimaryIndexTest();
}
@@ -131,7 +132,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index a043675..51dbb69 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -28,12 +28,13 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -76,7 +77,7 @@
// field, type and key declarations for primary index
private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private int primaryKeyFieldCount = 1;
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -98,13 +99,13 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
- primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
loadPrimaryIndexTest();
}
@@ -131,7 +132,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index 51d3db1..a804bae 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -27,11 +27,12 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -73,7 +74,7 @@
// field, type and key declarations for primary index
private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private int primaryKeyFieldCount = 1;
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -95,13 +96,13 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
- primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
loadPrimaryIndexTest();
}
@@ -128,7 +129,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 3b6b202..a16bbe4 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -28,12 +28,13 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -79,7 +80,7 @@
// field, type and key declarations for primary index
private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private int primaryKeyFieldCount = 1;
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -100,7 +101,7 @@
// field, type and key declarations for secondary indexes
private int secondaryFieldCount = 2;
- private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
+ private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
private int secondaryKeyFieldCount = 2;
private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
@@ -121,19 +122,19 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
- primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
// field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+ secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
loadPrimaryIndexTest();
loadSecondaryIndexTest();
@@ -162,7 +163,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
@@ -211,7 +213,8 @@
// sort based on secondary keys
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, primaryRecDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ primaryRecDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
// load secondary index
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 1e070a8..ef1c6f6 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -28,12 +28,13 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -62,308 +63,238 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-public class BTreeSecondaryIndexSearchOperatorTest extends
- AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+public class BTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
- private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary index
- private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private int primaryKeyFieldCount = 1;
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- primaryTypeTraits);
- private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory);
- private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory);
+ // field, type and key declarations for primary index
+ private int primaryFieldCount = 6;
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+ private int primaryKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
- private static String primaryBtreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryBtreeName;
+ private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
- private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- // field, type and key declarations for secondary indexes
- private int secondaryFieldCount = 2;
- private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
- private int secondaryKeyFieldCount = 2;
- private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
- private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- secondaryTypeTraits);
- private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- secondaryTupleWriterFactory);
- private ITreeIndexFrameFactory secondaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- secondaryTupleWriterFactory);
+ // field, type and key declarations for secondary indexes
+ private int secondaryFieldCount = 2;
+ private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+ private int secondaryKeyFieldCount = 2;
+ private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
+ secondaryTypeTraits);
+ private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ secondaryTupleWriterFactory);
+ private ITreeIndexFrameFactory secondaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(secondaryTupleWriterFactory);
- private static String secondaryBtreeName = "secondary"
- + simpleDateFormat.format(new Date());
- private static String secondaryFileName = System
- .getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
+ private static String secondaryBtreeName = "secondary" + simpleDateFormat.format(new Date());
+ private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
- private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- secondaryFileName))) });
+ private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
- private RecordDescriptor secondaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary index
- primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary index
+ primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
- // field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ // field, type and key declarations for secondary indexes
+ secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+ secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
- loadPrimaryIndexTest();
- loadSecondaryIndexTest();
- }
+ loadPrimaryIndexTest();
+ loadSecondaryIndexTest();
+ }
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/tpch0.001/orders-part1.tbl"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/tpch0.001/orders-part1.tbl"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, indexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeBulkLoad, NC1_ID);
+ int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- primaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
- spec.addRoot(primaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(primaryBtreeBulkLoad);
+ runTest(spec);
+ }
- public void loadSecondaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadSecondaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- // build dummy tuple containing nothing
- ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
- DataOutput dos = tb.getDataOutput();
+ // build dummy tuple containing nothing
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+ DataOutput dos = tb.getDataOutput();
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+ tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- int[] lowKeyFields = null; // - infinity
- int[] highKeyFields = null; // + infinity
+ int[] lowKeyFields = null; // - infinity
+ int[] highKeyFields = null; // + infinity
- // scan primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- indexRegistryProvider, primaryBtreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories, true,
- lowKeyFields, highKeyFields, true, true, dataflowHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeSearchOp, NC1_ID);
+ // scan primary index
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+ highKeyFields, true, true, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- // sort based on secondary keys
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 3, 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- primaryRecDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ // sort based on secondary keys
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ primaryRecDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- // load secondary index
- int[] fieldPermutation = { 3, 0 };
- TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, indexRegistryProvider,
- secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
- secondaryLeafFrameFactory, secondaryTypeTraits,
- secondaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryBtreeBulkLoad, NC1_ID);
+ // load secondary index
+ int[] fieldPermutation = { 3, 0 };
+ TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
+ secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- primaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeSearchOp, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- secondaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, secondaryBtreeBulkLoad, 0);
- spec.addRoot(secondaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(secondaryBtreeBulkLoad);
+ runTest(spec);
+ }
- @Test
- public void searchSecondaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ @Test
+ public void searchSecondaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- // build tuple containing search keys (only use the first key as search
- // key)
- ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
- DataOutput dos = tb.getDataOutput();
+ // build tuple containing search keys (only use the first key as search
+ // key)
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+ DataOutput dos = tb.getDataOutput();
- tb.reset();
- // low key
- UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
- tb.addFieldEndOffset();
- // high key
- UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ // low key
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
+ tb.addFieldEndOffset();
+ // high key
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
+ tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- int[] secondaryLowKeyFields = { 0 };
- int[] secondaryHighKeyFields = { 1 };
+ int[] secondaryLowKeyFields = { 0 };
+ int[] secondaryHighKeyFields = { 1 };
- // search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, secondaryRecDesc, storageManager,
- indexRegistryProvider, secondaryBtreeSplitProvider,
- secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
- secondaryTypeTraits, secondaryComparatorFactories, true,
- secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryBtreeSearchOp, NC1_ID);
+ // search secondary index
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
+ secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryTypeTraits,
+ secondaryComparatorFactories, true, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
- int[] primaryLowKeyFields = { 1 }; // second field from the tuples
- // coming from secondary index
- int[] primaryHighKeyFields = { 1 }; // second field from the tuples
- // coming from secondary index
+ int[] primaryLowKeyFields = { 1 }; // second field from the tuples
+ // coming from secondary index
+ int[] primaryHighKeyFields = { 1 }; // second field from the tuples
+ // coming from secondary index
- // search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- indexRegistryProvider, primaryBtreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories, true,
- primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeSearchOp, NC1_ID);
+ // search primary index
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
createTempFile().getAbsolutePath()) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- secondaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeSearchOp, 0, printer, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
- spec.addRoot(printer);
- runTest(spec);
- }
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
- File secondary = new File(secondaryFileName);
- secondary.deleteOnExit();
- }
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
+ File secondary = new File(secondaryFileName);
+ secondary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index abedf4a..fac141c 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -28,7 +28,7 @@
import org.junit.Rule;
import org.junit.rules.TemporaryFolder;
-import edu.uci.ics.hyracks.api.client.HyracksLocalConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
@@ -61,7 +61,10 @@
@BeforeClass
public static void init() throws Exception {
CCConfig ccConfig = new CCConfig();
- ccConfig.port = 39001;
+ ccConfig.clientNetIpAddress = "127.0.0.1";
+ ccConfig.clientNetPort = 39000;
+ ccConfig.clusterNetIpAddress = "127.0.0.1";
+ ccConfig.clusterNetPort = 39001;
ccConfig.profileDumpPeriod = 10000;
File outDir = new File("target/ClusterController");
outDir.mkdirs();
@@ -75,6 +78,7 @@
NCConfig ncConfig1 = new NCConfig();
ncConfig1.ccHost = "localhost";
ncConfig1.ccPort = 39001;
+ ncConfig1.clusterNetIPAddress = "127.0.0.1";
ncConfig1.dataIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
@@ -83,12 +87,13 @@
NCConfig ncConfig2 = new NCConfig();
ncConfig2.ccHost = "localhost";
ncConfig2.ccPort = 39001;
+ ncConfig2.clusterNetIPAddress = "127.0.0.1";
ncConfig2.dataIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
- hcc = new HyracksLocalConnection(cc);
+ hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
hcc.createApplication("test", null);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
index 042c919..1c5c014 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
@@ -27,8 +27,9 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -59,61 +60,40 @@
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MinMaxStringFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
/**
*
*/
public class AggregationTests extends AbstractIntegrationTest {
- final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
- "data/tpch0.001/lineitem.tbl"))) });
+ final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
+ new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
- final RecordDescriptor desc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
- FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE, }, '|');
+ final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+ UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+ FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, }, '|');
- private AbstractSingleActivityOperatorDescriptor getPrinter(
- JobSpecification spec, String prefix) throws IOException {
+ private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+ throws IOException {
- AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
- spec, new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC1_ID, createTempFile()
- .getAbsolutePath()),
- new FileSplit(NC2_ID, createTempFile()
- .getAbsolutePath()) }), "\t");
+ AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
+ new ConstantFileSplitProvider(new FileSplit[] {
+ new FileSplit(NC1_ID, createTempFile().getAbsolutePath()),
+ new FileSplit(NC2_ID, createTempFile().getAbsolutePath()) }), "\t");
return printer;
}
@@ -122,49 +102,38 @@
public void singleKeySumInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -177,64 +146,34 @@
public void singleKeySumPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
-
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
-
- int[] keyFields = new int[] { 0 };
-
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 4,
- keyFields,
- null,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-
- spec.connect(conn0, csvScanner, 0, sorter, 0);
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ int[] keyFields = new int[] { 0 };
+
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(conn1, sorter, 0, grouper, 0);
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
+ spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -247,58 +186,41 @@
public void singleKeySumExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new IntSumFieldAggregatorFactory(2, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- tableSize), true);
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new IntSumFieldAggregatorFactory(2, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -311,51 +233,38 @@
public void singleKeyAvgInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -368,66 +277,34 @@
public void singleKeyAvgPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 4,
- keyFields,
- null,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- desc);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-
- spec.connect(conn0, csvScanner, 0, sorter, 0);
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
+ spec.connect(conn1, csvScanner, 0, grouper, 0);
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec);
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(conn1, sorter, 0, grouper, 0);
-
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -440,61 +317,43 @@
public void singleKeyAvgExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new CountFieldAggregatorFactory(false),
- new AvgFieldGroupAggregatorFactory(1, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+ new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(2, false),
- new AvgFieldMergeAggregatorFactory(3, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- tableSize), true);
+ new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -507,49 +366,38 @@
public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -562,64 +410,34 @@
public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
-
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
-
- int[] keyFields = new int[] { 0 };
-
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 4,
- keyFields,
- null,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-
- spec.connect(conn0, csvScanner, 0, sorter, 0);
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec);
+ int[] keyFields = new int[] { 0 };
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(conn1, sorter, 0, grouper, 0);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
+ spec.connect(conn1, csvScanner, 0, grouper, 0);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -632,60 +450,42 @@
public void singleKeyMinMaxStringExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(15,
- true, true) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(2, true,
- true) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- tableSize), true);
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -698,53 +498,39 @@
public void multiKeySumInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -757,67 +543,35 @@
public void multiKeySumPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
-
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
-
- int[] keyFields = new int[] { 8, 0 };
-
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 4,
- keyFields,
- null,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE},
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE}));
-
- spec.connect(conn0, csvScanner, 0, sorter, 0);
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields, new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ int[] keyFields = new int[] { 8, 0 };
+
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(conn1, sorter, 0, grouper, 0);
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+ spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -830,64 +584,43 @@
public void multiKeySumExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- int[] keys = new int[] { 0, 1 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(keyFields,
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- new MultiFieldsAggregatorFactory(keys,
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+ new IntSumFieldAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -900,55 +633,40 @@
public void multiKeyAvgInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -961,69 +679,36 @@
public void multiKeyAvgPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
-
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
-
- int[] keyFields = new int[] { 8, 0 };
-
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 4,
- keyFields,
- null,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE},
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE}));
-
- spec.connect(conn0, csvScanner, 0, sorter, 0);
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields, new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec);
+ int[] keyFields = new int[] { 8, 0 };
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(conn1, sorter, 0, grouper, 0);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgInmemGroupTest");
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+ spec.connect(conn1, csvScanner, 0, grouper, 0);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1036,66 +721,46 @@
public void multiKeyAvgExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new CountFieldAggregatorFactory(false),
- new AvgFieldGroupAggregatorFactory(1, false) }),
- new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+ new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
new IntSumFieldAggregatorFactory(3, false),
- new AvgFieldMergeAggregatorFactory(4, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1108,53 +773,39 @@
public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1167,67 +818,35 @@
public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
-
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
-
- int[] keyFields = new int[] { 8, 0 };
-
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 4,
- keyFields,
- null,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE},
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE}));
-
- spec.connect(conn0, csvScanner, 0, sorter, 0);
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields, new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec);
+ int[] keyFields = new int[] { 8, 0 };
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(conn1, sorter, 0, grouper, 0);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringPreClusterGroupTest");
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+ spec.connect(conn1, csvScanner, 0, grouper, 0);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1240,65 +859,44 @@
public void multiKeyMinMaxStringExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(15,
- true, true) }),
- new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
- new MinMaxStringFieldAggregatorFactory(3, true,
- true) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+ new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(2, false),
+ new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index 1c96755..ce72ec5 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -27,9 +27,10 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -69,7 +70,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -77,19 +79,19 @@
PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
spec,
new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID);
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
@@ -100,7 +102,8 @@
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, sorter, 0);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -108,7 +111,8 @@
IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn3, group, 0, sorter2, 0);
IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
@@ -138,7 +142,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -146,19 +151,19 @@
PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
spec,
new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID, NC2_ID);
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
@@ -169,7 +174,8 @@
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, sorter, 0);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -177,7 +183,8 @@
IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn3, group, 0, sorter2, 0);
IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
@@ -207,7 +214,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 3, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -215,19 +223,19 @@
PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
spec,
new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID, NC2_ID);
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
@@ -238,7 +246,8 @@
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, sorter, 0);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -246,7 +255,8 @@
IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn3, group, 0, sorter2, 0);
IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 55a0c1c..9355110 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -26,8 +26,9 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -104,7 +105,8 @@
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, ordScanner, 0, printer, 0);
spec.addRoot(printer);
@@ -140,7 +142,8 @@
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { IntegerBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(IntegerPointable.FACTORY) }));
spec.connect(conn1, ordScanner, 0, printer, 0);
spec.addRoot(printer);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 57888c8..2c3fddf 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -26,8 +26,9 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -68,7 +69,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -78,10 +80,12 @@
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new int[] { 1 }, new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }),
- sorter, 0, printer, 0);
+ spec.connect(
+ new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+ new int[] { 1 }, new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), new int[] { 1 },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+ .of(UTF8StringPointable.FACTORY) }), sorter, 0, printer, 0);
runTest(spec);
}
@@ -110,8 +114,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 4, new int[] { 1, 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -121,11 +125,15 @@
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- new int[] { 1, 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
- UTF8StringBinaryHashFunctionFactory.INSTANCE }), new int[] { 1, 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
- UTF8StringBinaryComparatorFactory.INSTANCE }), sorter, 0, printer, 0);
+ spec.connect(
+ new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(new int[] {
+ 1, 0 }, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
+ new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
+ printer, 0);
runTest(spec);
}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 2fbf6d2..61d4696 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -32,8 +32,9 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -138,9 +139,13 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
- new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+ InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+ spec,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -211,10 +216,17 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 4, 10, 200, 1.2,
- new int[] { 1 }, new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+ GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(
+ spec,
+ 4,
+ 10,
+ 200,
+ 1.2,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -285,10 +297,17 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
- new int[] { 1 }, new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+ HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(
+ spec,
+ 5,
+ 20,
+ 200,
+ 1.2,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -359,15 +378,18 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFields().length];
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
for (int j = 0; j < nullWriterFactories.length; j++) {
nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
}
- InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 0 },
- new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, true,
- nullWriterFactories, 128);
+ InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+ spec,
+ new int[] { 0 },
+ new int[] { 1 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, true, nullWriterFactories, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -438,16 +460,22 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFields().length];
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
for (int j = 0; j < nullWriterFactories.length; j++) {
nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
}
- GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
- new int[] { 0 }, new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, true,
- nullWriterFactories);
+ GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(
+ spec,
+ 5,
+ 20,
+ 200,
+ 1.2,
+ new int[] { 0 },
+ new int[] { 1 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, true, nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -518,16 +546,22 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFields().length];
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
for (int j = 0; j < nullWriterFactories.length; j++) {
nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
}
- HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
- new int[] { 0 }, new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, true,
- nullWriterFactories);
+ HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(
+ spec,
+ 5,
+ 20,
+ 200,
+ 1.2,
+ new int[] { 0 },
+ new int[] { 1 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, true, nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -600,9 +634,13 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
- InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
- new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+ InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+ spec,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -612,12 +650,14 @@
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -679,10 +719,17 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
- GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 3, 20, 100, 1.2,
- new int[] { 1 }, new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+ GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(
+ spec,
+ 3,
+ 20,
+ 100,
+ 1.2,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -692,12 +739,14 @@
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -759,10 +808,17 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
- HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 3, 20, 100, 1.2,
- new int[] { 1 }, new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+ HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(
+ spec,
+ 3,
+ 20,
+ 100,
+ 1.2,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -772,12 +828,14 @@
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -839,9 +897,13 @@
UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
- InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
- new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+ InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+ spec,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, 128);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -851,12 +913,14 @@
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -924,9 +988,13 @@
MaterializingOperatorDescriptor custMat = new MaterializingOperatorDescriptor(spec, custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custMat, NC1_ID, NC2_ID);
- InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
- new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+ InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+ spec,
+ new int[] { 1 },
+ new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -936,12 +1004,14 @@
IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(ordPartConn, ordScanner, 0, ordMat, 0);
IConnectorDescriptor custPartConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(custPartConn, custScanner, 0, custMat, 0);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 135f1fe..6411390 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -30,7 +30,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -163,7 +164,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 4);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -237,7 +238,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 5);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -311,7 +312,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 6);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
index 6440db3..298c146 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -27,14 +27,15 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -99,7 +100,7 @@
// Primary BTree index.
private int primaryFieldCount = 2;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private int primaryKeyFieldCount = 1;
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -110,12 +111,12 @@
IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
// Inverted index BTree dictionary.
- private ITypeTrait[] tokenTypeTraits = new ITypeTrait[1];
+ private ITypeTraits[] tokenTypeTraits = new ITypeTraits[1];
private IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[1];
// Inverted index stuff.
private int invListElementFieldCount = 1;
- private ITypeTrait[] invListsTypeTraits = new ITypeTrait[invListElementFieldCount];
+ private ITypeTraits[] invListsTypeTraits = new ITypeTraits[invListElementFieldCount];
private IBinaryComparatorFactory[] invListsComparatorFactories = new IBinaryComparatorFactory[invListElementFieldCount];
private RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -130,17 +131,17 @@
@Before
public void setup() throws Exception {
// Field declarations and comparators for primary BTree index.
- primaryTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
// Field declarations and comparators for tokens.
- tokenTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- tokenComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ tokenTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ tokenComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
// Field declarations and comparators for inverted lists.
- invListsTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
- invListsComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+ invListsTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ invListsComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
loadPrimaryIndex();
printPrimaryIndex();
@@ -232,8 +233,8 @@
private IOperatorDescriptor createExternalSortOp(JobSpecification spec, int[] sortFields,
RecordDescriptor outputRecDesc) {
ExternalSortOperatorDescriptor externalSortOp = new ExternalSortOperatorDescriptor(spec, 1000, sortFields,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
- IntegerBinaryComparatorFactory.INSTANCE }, outputRecDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, outputRecDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, externalSortOp, NC1_ID);
return externalSortOp;
}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 74dc4a8..0c0df42 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -28,12 +28,14 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -80,7 +82,7 @@
// field, type and key declarations for primary R-tree index
private int primaryFieldCount = 5;
private int primaryKeyFieldCount = 4;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
@@ -103,18 +105,18 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary R-tree index
- primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
primaryComparatorFactories[1] = primaryComparatorFactories[0];
primaryComparatorFactories[2] = primaryComparatorFactories[0];
primaryComparatorFactories[3] = primaryComparatorFactories[0];
IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
+ .createPrimitiveValueProviderFactories(primaryComparatorFactories.length, DoublePointable.FACTORY);
primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
primaryValueProviderFactories);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index 62fc150..a4912e8 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -27,11 +27,12 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -77,7 +78,7 @@
// field, type and key declarations for primary R-tree index
private int primaryFieldCount = 5;
private int primaryKeyFieldCount = 4;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
@@ -100,18 +101,18 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary R-tree index
- primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+ primaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+ primaryTypeTraits[4] = DoublePointable.TYPE_TRAITS;
+ primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
primaryComparatorFactories[1] = primaryComparatorFactories[0];
primaryComparatorFactories[2] = primaryComparatorFactories[0];
primaryComparatorFactories[3] = primaryComparatorFactories[0];
IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
+ .createPrimitiveValueProviderFactories(primaryComparatorFactories.length, DoublePointable.FACTORY);
primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
primaryValueProviderFactories);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 300a726..025f675 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -28,13 +28,14 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -88,7 +89,7 @@
// field, type and key declarations for primary B-tree index
private int primaryBTreeFieldCount = 10;
- private ITypeTrait[] primaryBTreeTypeTraits = new ITypeTrait[primaryBTreeFieldCount];
+ private ITypeTraits[] primaryBTreeTypeTraits = new ITypeTraits[primaryBTreeFieldCount];
private int primaryBTreeKeyFieldCount = 1;
private IBinaryComparatorFactory[] primaryBTreeComparatorFactories = new IBinaryComparatorFactory[primaryBTreeKeyFieldCount];
private TypeAwareTupleWriterFactory primaryBTreeTupleWriterFactory = new TypeAwareTupleWriterFactory(
@@ -113,7 +114,7 @@
// field, type and key declarations for secondary indexes
private int secondaryFieldCount = 5;
- private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
+ private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
private int secondaryKeyFieldCount = 4;
private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
private IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = new IPrimitiveValueProviderFactory[secondaryKeyFieldCount];
@@ -138,25 +139,25 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary B-tree index
- primaryBTreeTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryBTreeTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryBTreeTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryBTreeTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryBTreeTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryBTreeTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
- primaryBTreeTypeTraits[6] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryBTreeTypeTraits[7] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryBTreeTypeTraits[8] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryBTreeTypeTraits[9] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- primaryBTreeComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ primaryBTreeTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[6] = DoublePointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[7] = DoublePointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[8] = DoublePointable.TYPE_TRAITS;
+ primaryBTreeTypeTraits[9] = DoublePointable.TYPE_TRAITS;
+ primaryBTreeComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
// field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- secondaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- secondaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- secondaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
- secondaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
- secondaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+ secondaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+ secondaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+ secondaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+ secondaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+ secondaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+ secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
secondaryComparatorFactories[1] = secondaryComparatorFactories[0];
secondaryComparatorFactories[2] = secondaryComparatorFactories[0];
secondaryComparatorFactories[3] = secondaryComparatorFactories[0];
@@ -166,7 +167,7 @@
secondaryValueProviderFactories[3] = secondaryValueProviderFactories[0];
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(secondaryComparatorFactories);
+ .createPrimitiveValueProviderFactories(secondaryComparatorFactories.length, DoublePointable.FACTORY);
secondaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(secondaryTupleWriterFactory,
secondaryValueProviderFactories);
@@ -203,7 +204,8 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index 8db3f48..66f7efc 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -19,7 +19,7 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
@@ -31,8 +31,9 @@
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -53,11 +54,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
@@ -98,7 +97,7 @@
@Option(name = "-out-plain", usage = "Whether to output plain text (default: true)", required = false)
public boolean outPlain = true;
-
+
@Option(name = "-algo", usage = "The algorithm to be used", required = true)
public int algo;
}
@@ -111,17 +110,15 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host,
- options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job;
for (int i = 0; i < 6; i++) {
long start = System.currentTimeMillis();
- job = createJob(parseFileSplits(options.inFileSplits),
- parseFileSplits(options.outFileSplits, i),
- options.htSize, options.sbSize, options.framesLimit,
- options.sortOutput, options.algo, options.outPlain);
+ job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits, i),
+ options.htSize, options.sbSize, options.framesLimit, options.sortOutput, options.algo,
+ options.outPlain);
System.out.print(i + "\t" + (System.currentTimeMillis() - start));
start = System.currentTimeMillis();
@@ -139,11 +136,9 @@
String s = splits[i].trim();
int idx = s.indexOf(':');
if (idx < 0) {
- throw new IllegalArgumentException("File split " + s
- + " not well formed");
+ throw new IllegalArgumentException("File split " + s + " not well formed");
}
- fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(
- new File(s.substring(idx + 1))));
+ fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
}
return fSplits;
}
@@ -155,70 +150,46 @@
String s = splits[i].trim();
int idx = s.indexOf(':');
if (idx < 0) {
- throw new IllegalArgumentException("File split " + s
- + " not well formed");
+ throw new IllegalArgumentException("File split " + s + " not well formed");
}
- fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(
- new File(s.substring(idx + 1) + "_" + count)));
+ fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1) + "_"
+ + count)));
}
return fSplits;
}
- private static JobSpecification createJob(FileSplit[] inSplits,
- FileSplit[] outSplits, int htSize, int sbSize, int framesLimit,
- boolean sortOutput, int alg, boolean outPlain) {
+ private static JobSpecification createJob(FileSplit[] inSplits, FileSplit[] outSplits, int htSize, int sbSize,
+ int framesLimit, boolean sortOutput, int alg, boolean outPlain) {
JobSpecification spec = new JobSpecification();
- IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(
- inSplits);
+ IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(inSplits);
- RecordDescriptor inDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor inDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor fileScanner = new FileScanOperatorDescriptor(
- spec, splitsProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- FloatParserFactory.INSTANCE,
- FloatParserFactory.INSTANCE,
- FloatParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE, }, '|'),
- inDesc);
+ FileScanOperatorDescriptor fileScanner = new FileScanOperatorDescriptor(spec, splitsProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+ FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, }, '|'), inDesc);
createPartitionConstraint(spec, fileScanner, inSplits);
// Output: each unique string with an integer count
- RecordDescriptor outDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- IntegerSerializerDeserializer.INSTANCE,
- // IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE,
+ // IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE });
// Specify the grouping key, which will be the string extracted during
// the scan.
@@ -229,156 +200,126 @@
AbstractOperatorDescriptor grouper;
switch (alg) {
- case 0: // new external hash graph
- grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keys,
- framesLimit,
- new IBinaryComparatorFactory[] {
- // IntegerBinaryComparatorFactory.INSTANCE,
- IntegerBinaryComparatorFactory.INSTANCE },
- new IntegerNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(
- false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(
- keys.length, false) }), outDesc,
- new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }),
- htSize), false);
-
- createPartitionConstraint(spec, grouper, outSplits);
+ case 0: // new external hash graph
+ grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(spec, keys,
+ framesLimit, new IBinaryComparatorFactory[] {
+ // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(keys.length,
+ false) }), outDesc, new HashSpillableTableFactory(
+ new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }), htSize), false);
- // Connect scanner with the grouper
- IConnectorDescriptor scanGroupConnDef2 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(scanGroupConnDef2, fileScanner, 0, grouper, 0);
-
- break;
- case 1: // External-sort + new-precluster
- ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(
- spec, framesLimit, keys, new IBinaryComparatorFactory[] {
- // IntegerBinaryComparatorFactory.INSTANCE,
- IntegerBinaryComparatorFactory.INSTANCE }, inDesc);
- createPartitionConstraint(spec, sorter2, inSplits);
+ createPartitionConstraint(spec, grouper, outSplits);
- // Connect scan operator with the sorter
- IConnectorDescriptor scanSortConn2 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(scanSortConn2, fileScanner, 0, sorter2, 0);
+ // Connect scanner with the grouper
+ IConnectorDescriptor scanGroupConnDef2 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+ spec.connect(scanGroupConnDef2, fileScanner, 0, grouper, 0);
- grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keys,
- new IBinaryComparatorFactory[] {
- // IntegerBinaryComparatorFactory.INSTANCE,
- IntegerBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- outDesc);
+ break;
+ case 1: // External-sort + new-precluster
+ ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(spec, framesLimit, keys,
+ new IBinaryComparatorFactory[] {
+ // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, inDesc);
+ createPartitionConstraint(spec, sorter2, inSplits);
- createPartitionConstraint(spec, grouper, outSplits);
+ // Connect scan operator with the sorter
+ IConnectorDescriptor scanSortConn2 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+ spec.connect(scanSortConn2, fileScanner, 0, sorter2, 0);
- // Connect sorter with the pre-cluster
- OneToOneConnectorDescriptor sortGroupConn2 = new OneToOneConnectorDescriptor(
- spec);
- spec.connect(sortGroupConn2, sorter2, 0, grouper, 0);
- break;
- case 2: // Inmem
- grouper = new HashGroupOperatorDescriptor(
- spec,
- keys,
- new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] {
- // IntegerBinaryComparatorFactory.INSTANCE,
- IntegerBinaryComparatorFactory.INSTANCE },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- outDesc, htSize);
+ grouper = new edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor(spec, keys,
+ new IBinaryComparatorFactory[] {
+ // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
+ outDesc);
- createPartitionConstraint(spec, grouper, outSplits);
+ createPartitionConstraint(spec, grouper, outSplits);
- // Connect scanner with the grouper
- IConnectorDescriptor scanConn2 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(scanConn2, fileScanner, 0, grouper, 0);
- break;
- default:
- grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(
- spec,
- keys,
- framesLimit,
- new IBinaryComparatorFactory[] {
- // IntegerBinaryComparatorFactory.INSTANCE,
- IntegerBinaryComparatorFactory.INSTANCE },
- new IntegerNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(
- false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(
- keys.length, false) }), outDesc,
- new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }),
- htSize), false);
-
- createPartitionConstraint(spec, grouper, outSplits);
+ // Connect sorter with the pre-cluster
+ OneToOneConnectorDescriptor sortGroupConn2 = new OneToOneConnectorDescriptor(spec);
+ spec.connect(sortGroupConn2, sorter2, 0, grouper, 0);
+ break;
+ case 2: // Inmem
+ grouper = new HashGroupOperatorDescriptor(spec, keys, new FieldHashPartitionComputerFactory(keys,
+ new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] {
+ // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
+ outDesc, htSize);
- // Connect scanner with the grouper
- IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] {
- // IntegerBinaryHashFunctionFactory.INSTANCE,
- IntegerBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
+ createPartitionConstraint(spec, grouper, outSplits);
+
+ // Connect scanner with the grouper
+ IConnectorDescriptor scanConn2 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+ spec.connect(scanConn2, fileScanner, 0, grouper, 0);
+ break;
+ default:
+ grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(spec, keys,
+ framesLimit, new IBinaryComparatorFactory[] {
+ // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(keys.length,
+ false) }), outDesc, new HashSpillableTableFactory(
+ new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }), htSize), false);
+
+ createPartitionConstraint(spec, grouper, outSplits);
+
+ // Connect scanner with the grouper
+ IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+ // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+ spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
}
- IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(
- outSplits);
+ IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
AbstractSingleActivityOperatorDescriptor writer;
if (outPlain)
- writer = new PlainFileWriterOperatorDescriptor(spec,
- outSplitProvider, "|");
+ writer = new PlainFileWriterOperatorDescriptor(spec, outSplitProvider, "|");
else
- writer = new FrameFileWriterOperatorDescriptor(spec,
- outSplitProvider);
+ writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
createPartitionConstraint(spec, writer, outSplits);
- IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(
- spec);
+ IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(spec);
spec.connect(groupOutConn, grouper, 0, writer, 0);
spec.addRoot(writer);
return spec;
}
- private static void createPartitionConstraint(JobSpecification spec,
- IOperatorDescriptor op, FileSplit[] splits) {
+ private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
String[] parts = new String[splits.length];
for (int i = 0; i < splits.length; ++i) {
parts[i] = splits[i].getNodeName();
}
- PartitionConstraintHelper
- .addAbsoluteLocationConstraint(spec, op, parts);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
}
}
\ No newline at end of file
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index 8cac822..1708259 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -20,7 +20,7 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
@@ -33,8 +33,9 @@
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
@@ -62,7 +63,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -94,7 +95,7 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
JobSpecification job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits),
options.algo, options.htSize, options.sbSize, options.format);
@@ -140,19 +141,25 @@
IOperatorDescriptor gBy;
int[] keys = new int[] { 0 };
if ("hash".equalsIgnoreCase(algo)) {
- gBy = new HashGroupOperatorDescriptor(spec, keys, new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ gBy = new HashGroupOperatorDescriptor(
+ spec,
+ keys,
+ new FieldHashPartitionComputerFactory(keys,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
groupResultDesc, htSize);
createPartitionConstraint(spec, gBy, outSplits);
IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(scanGroupConn, wordScanner, 0, gBy, 0);
} else {
- IBinaryComparatorFactory[] cfs = new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE };
+ IBinaryComparatorFactory[] cfs = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+ .of(UTF8StringPointable.FACTORY) };
IOperatorDescriptor sorter = "memsort".equalsIgnoreCase(algo) ? new InMemorySortOperatorDescriptor(spec,
keys, new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc)
: new ExternalSortOperatorDescriptor(spec, sbSize, keys,
@@ -161,11 +168,14 @@
IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(scanSortConn, wordScanner, 0, sorter, 0);
- gBy = new PreclusteredGroupOperatorDescriptor(spec, keys,
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ gBy = new PreclusteredGroupOperatorDescriptor(
+ spec,
+ keys,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
groupResultDesc);
diff --git a/hyracks-examples/text-example/texthelper/pom.xml b/hyracks-examples/text-example/texthelper/pom.xml
index 60c423d..b0bfcef 100644
--- a/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks-examples/text-example/texthelper/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.text</groupId>
<artifactId>texthelper</artifactId>
- <version>0.2.0-SNAPSHOT</version>
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
@@ -23,6 +22,11 @@
<version>0.2.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/hyracks-examples/tpch-example/tpchapp/pom.xml b/hyracks-examples/tpch-example/tpchapp/pom.xml
index c65aaf5..05256a3 100644
--- a/hyracks-examples/tpch-example/tpchapp/pom.xml
+++ b/hyracks-examples/tpch-example/tpchapp/pom.xml
@@ -2,8 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
<artifactId>tpchapp</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>tpch-example</artifactId>
@@ -83,5 +81,10 @@
<version>0.2.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks-examples/tpch-example/tpchclient/pom.xml
index c7a138a..d9b5fce 100644
--- a/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -2,8 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
<artifactId>tpchclient</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>tpch-example</artifactId>
@@ -17,6 +15,11 @@
<version>0.2.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 9e88f8c..72533e7 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -20,7 +20,7 @@
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
@@ -38,8 +38,9 @@
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -68,7 +69,7 @@
public String host;
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)", required = false)
- public int port = 1099;
+ public int port = 1098;
@Option(name = "-app", usage = "Hyracks Application name", required = true)
public String app;
@@ -117,24 +118,16 @@
CmdLineParser parser = new CmdLineParser(options);
parser.parseArgument(args);
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host,
- options.port);
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
- JobSpecification job = createJob(
- parseFileSplits(options.inFileCustomerSplits),
- parseFileSplits(options.inFileOrderSplits),
- parseFileSplits(options.outFileSplits),
- options.numJoinPartitions, options.algo,
- options.graceInputSize, options.graceRecordsPerFrame,
- options.graceFactor, options.memSize, options.tableSize,
- options.hasGroupBy);
+ JobSpecification job = createJob(parseFileSplits(options.inFileCustomerSplits),
+ parseFileSplits(options.inFileOrderSplits), parseFileSplits(options.outFileSplits),
+ options.numJoinPartitions, options.algo, options.graceInputSize, options.graceRecordsPerFrame,
+ options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(
- options.app,
- job,
- options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet
- .noneOf(JobFlag.class));
+ JobId jobId = hcc.createJob(options.app, job,
+ options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
@@ -148,104 +141,65 @@
String s = splits[i].trim();
int idx = s.indexOf(':');
if (idx < 0) {
- throw new IllegalArgumentException("File split " + s
- + " not well formed");
+ throw new IllegalArgumentException("File split " + s + " not well formed");
}
- fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(
- new File(s.substring(idx + 1))));
+ fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
}
return fSplits;
}
- private static JobSpecification createJob(FileSplit[] customerSplits,
- FileSplit[] orderSplits, FileSplit[] resultSplits,
- int numJoinPartitions, String algo, int graceInputSize,
- int graceRecordsPerFrame, double graceFactor, int memSize,
- int tableSize, boolean hasGroupBy) throws HyracksDataException {
+ private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
+ FileSplit[] resultSplits, int numJoinPartitions, String algo, int graceInputSize, int graceRecordsPerFrame,
+ double graceFactor, int memSize, int tableSize, boolean hasGroupBy) throws HyracksDataException {
JobSpecification spec = new JobSpecification();
- IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(
- customerSplits);
- RecordDescriptor custDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
+ RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(
- orderSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- RecordDescriptor custOrderJoinDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitsProvider,
- new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
createPartitionConstraint(spec, ordScanner, orderSplits);
- FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(
- spec, custSplitsProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- custDesc);
+ FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
createPartitionConstraint(spec, custScanner, customerSplits);
IOperatorDescriptor join;
if ("nestedloop".equalsIgnoreCase(algo)) {
- join = new NestedLoopJoinOperatorDescriptor(spec,
- new JoinComparatorFactory(
- UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- custOrderJoinDesc, memSize);
+ join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize);
} else if ("gracehash".equalsIgnoreCase(algo)) {
join = new GraceHashJoinOperatorDescriptor(
@@ -256,8 +210,9 @@
graceFactor,
new int[] { 0 },
new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
custOrderJoinDesc);
} else if ("hybridhash".equalsIgnoreCase(algo)) {
@@ -269,8 +224,9 @@
graceFactor,
new int[] { 0 },
new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
custOrderJoinDesc);
} else {
@@ -278,81 +234,71 @@
spec,
new int[] { 0 },
new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
custOrderJoinDesc, 6000000);
}
- PartitionConstraintHelper.addPartitionCountConstraint(spec, join,
- numJoinPartitions);
+ PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
- IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- new int[] { 1 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(new int[] { 1 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(ordJoinConn, ordScanner, 0, join, 1);
- IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- new int[] { 0 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 0);
IOperatorDescriptor endingOp = join;
if (hasGroupBy) {
- RecordDescriptor groupResultDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(
spec,
new int[] { 6 },
- new FieldHashPartitionComputerFactory(
- new int[] { 6 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+ new FieldHashPartitionComputerFactory(new int[] { 6 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
groupResultDesc, 16);
createPartitionConstraint(spec, gby, resultSplits);
- IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- new int[] { 6 },
- new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+ IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(new int[] { 6 },
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }));
spec.connect(joinGroupConn, join, 0, gby, 0);
endingOp = gby;
}
- IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(
- resultSplits);
- FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(
- spec, outSplitProvider);
+ IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(resultSplits);
+ FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
createPartitionConstraint(spec, writer, resultSplits);
- IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(
- spec);
+ IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(spec);
spec.connect(endingPrinterConn, endingOp, 0, writer, 0);
spec.addRoot(writer);
return spec;
}
- private static void createPartitionConstraint(JobSpecification spec,
- IOperatorDescriptor op, FileSplit[] splits) {
+ private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
String[] parts = new String[splits.length];
for (int i = 0; i < splits.length; ++i) {
parts[i] = splits[i].getNodeName();
}
- PartitionConstraintHelper
- .addAbsoluteLocationConstraint(spec, op, parts);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
}
static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -362,8 +308,7 @@
private final int pos0;
private final int pos1;
- public JoinComparatorFactory(IBinaryComparatorFactory bFactory,
- int pos0, int pos1) {
+ public JoinComparatorFactory(IBinaryComparatorFactory bFactory, int pos0, int pos1) {
this.bFactory = bFactory;
this.pos0 = pos0;
this.pos1 = pos1;
@@ -371,8 +316,7 @@
@Override
public ITuplePairComparator createTuplePairComparator() {
- return new JoinComparator(bFactory.createBinaryComparator(), pos0,
- pos1);
+ return new JoinComparator(bFactory.createBinaryComparator(), pos0, pos1);
}
}
@@ -382,16 +326,14 @@
private final int field0;
private final int field1;
- public JoinComparator(IBinaryComparator bComparator, int field0,
- int field1) {
+ public JoinComparator(IBinaryComparator bComparator, int field0, int field1) {
this.bComparator = bComparator;
this.field0 = field0;
this.field1 = field1;
}
@Override
- public int compare(IFrameTupleAccessor accessor0, int tIndex0,
- IFrameTupleAccessor accessor1, int tIndex1) {
+ public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1) {
int tStart0 = accessor0.getTupleStartOffset(tIndex0);
int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
@@ -406,9 +348,8 @@
int fEnd1 = accessor1.getFieldEndOffset(tIndex1, field1);
int fLen1 = fEnd1 - fStart1;
- int c = bComparator.compare(accessor0.getBuffer().array(), fStart0
- + fStartOffset0, fLen0, accessor1.getBuffer().array(),
- fStart1 + fStartOffset1, fLen1);
+ int c = bComparator.compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
+ .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
if (c != 0) {
return c;
}
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index 9e76a3e..81e9f84 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -5,7 +5,7 @@
import java.util.Properties;
import java.util.Set;
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -15,7 +15,7 @@
public class HyracksClient {
- private static HyracksRMIConnection connection;
+ private static HyracksConnection connection;
private static final String jobProfilingKey = "jobProfilingKey";
Set<String> systemLibs;
@@ -25,7 +25,7 @@
private void initialize(Properties properties) throws Exception {
String clusterController = (String) properties.get(ConfigurationConstants.clusterControllerHost);
- connection = new HyracksRMIConnection(clusterController, 1099);
+ connection = new HyracksConnection(clusterController, 1099);
systemLibs = new HashSet<String>();
for (String systemLib : ConfigurationConstants.systemLibs) {
String systemLibPath = properties.getProperty(systemLib);
diff --git a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
index 8ed7bb7..4faa619 100644
--- a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
+++ b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
@@ -27,8 +27,11 @@
@Option(name = "-n", required = false, usage = "Number of node controllers (default: 2)")
public int n = 2;
- @Option(name = "-cc-port", required = false, usage = "CC Port (default: 1099)")
- public int ccPort = 1099;
+ @Option(name = "-cc-client-net-port", required = false, usage = "CC Port (default: 1098)")
+ public int ccClientNetPort = 1098;
+
+ @Option(name = "-cc-cluster-net-port", required = false, usage = "CC Port (default: 1099)")
+ public int ccClusterNetPort = 1099;
@Option(name = "-cc-http-port", required = false, usage = "CC Port (default: 19001)")
public int ccHttpPort = 19001;
@@ -46,7 +49,10 @@
}
CCConfig ccConfig = new CCConfig();
- ccConfig.port = options.ccPort;
+ ccConfig.clusterNetIpAddress = "127.0.0.1";
+ ccConfig.clusterNetPort = options.ccClusterNetPort;
+ ccConfig.clientNetIpAddress = "127.0.0.1";
+ ccConfig.clientNetPort = options.ccClientNetPort;
ccConfig.httpPort = options.ccHttpPort;
HyracksCCProcess ccp = new HyracksCCProcess(ccConfig);
ccp.start();
@@ -56,7 +62,9 @@
HyracksNCProcess ncps[] = new HyracksNCProcess[options.n];
for (int i = 0; i < options.n; ++i) {
NCConfig ncConfig = new NCConfig();
- ncConfig.ccHost = "localhost";
+ ncConfig.ccHost = "127.0.0.1";
+ ncConfig.ccPort = options.ccClusterNetPort;
+ ncConfig.clusterNetIPAddress = "127.0.0.1";
ncConfig.nodeId = "nc" + i;
ncConfig.dataIPAddress = "127.0.0.1";
ncps[i] = new HyracksNCProcess(ncConfig);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index 8e88c18..4486205 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -21,7 +21,7 @@
import java.util.Comparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.btree.api.IPrefixSlotManager;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.FieldPrefixSlotManager;
@@ -41,9 +41,9 @@
// them
private int occurrenceThreshold;
- private ITypeTrait[] typeTraits;
+ private ITypeTraits[] typeTraits;
- public FieldPrefixCompressor(ITypeTrait[] typeTraits, float ratioThreshold, int occurrenceThreshold) {
+ public FieldPrefixCompressor(ITypeTraits[] typeTraits, float ratioThreshold, int occurrenceThreshold) {
this.typeTraits = typeTraits;
this.ratioThreshold = ratioThreshold;
this.occurrenceThreshold = occurrenceThreshold;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 4075152..6314260 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -44,7 +44,7 @@
public BTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index 2be1641..8cde863 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -4,7 +4,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -24,7 +24,7 @@
public BTreeUpdateSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
ITupleUpdaterFactory tupleUpdaterFactory) {
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 56e0774..0aba70b 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -19,7 +19,7 @@
import java.util.ArrayList;
import java.util.Collections;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -77,7 +77,7 @@
public BTreeFieldPrefixNSMLeafFrame(ITreeIndexTupleWriter tupleWriter) {
this.tupleWriter = tupleWriter;
this.frameTuple = new FieldPrefixTupleReference(tupleWriter.createTupleReference());
- ITypeTrait[] typeTraits = ((TypeAwareTupleWriter) tupleWriter).getTypeTraits();
+ ITypeTraits[] typeTraits = ((TypeAwareTupleWriter) tupleWriter).getTypeTraits();
this.framePrefixTuple = new FieldPrefixPrefixTupleReference(typeTraits);
this.slotManager = new FieldPrefixSlotManager();
this.compressor = new FieldPrefixCompressor(typeTraits, 0.001f, 2);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 57f1b79..4113e9f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -58,8 +58,6 @@
private final static int MAX_RESTARTS = 10;
private final static int rootPage = 1;
- private boolean created = false;
-
private final IFreePageManager freePageManager;
private final IBufferCache bufferCache;
private final ITreeIndexFrameFactory interiorFrameFactory;
@@ -86,15 +84,11 @@
public void create(int fileId) throws HyracksDataException {
treeLatch.writeLock().lock();
try {
- if (created) {
- return;
- }
ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
ITreeIndexMetaDataFrame metaFrame = freePageManager.getMetaDataFrameFactory().createFrame();
this.fileId = fileId;
freePageManager.init(metaFrame, rootPage);
initRoot(leafFrame, true);
- created = true;
} finally {
treeLatch.writeLock().unlock();
}
@@ -305,8 +299,9 @@
insertUpdateOrDelete(tuple, ctx);
}
- private void insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+ private boolean insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
ctx.leafFrame.setPage(node);
+ boolean restartOp = false;
int targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(tuple);
FrameOpSpaceStatus spaceStatus = ctx.leafFrame.hasSpaceInsert(tuple);
switch (spaceStatus) {
@@ -324,7 +319,7 @@
ctx.splitKey.reset();
break;
}
- case INSUFFICIENT_SPACE: {
+ case INSUFFICIENT_SPACE: {
// Try compressing the page first and see if there is space available.
boolean reCompressed = ctx.leafFrame.compress();
if (reCompressed) {
@@ -336,31 +331,34 @@
ctx.leafFrame.insert(tuple, targetTupleIndex);
ctx.splitKey.reset();
} else {
- performLeafSplit(pageId, tuple, ctx);
+ restartOp = performLeafSplit(pageId, tuple, ctx);
}
break;
}
}
node.releaseWriteLatch();
bufferCache.unpin(node);
+ return restartOp;
}
- private void performLeafSplit(int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
- int rightSiblingPageId = ctx.leafFrame.getNextLeaf();
+ private boolean performLeafSplit(int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+ // Lock is released in unsetSmPages(), after sm has fully completed.
+ if (!treeLatch.writeLock().tryLock()) {
+ return true;
+ }
+ int rightSiblingPageId = ctx.leafFrame.getNextLeaf();
ICachedPage rightSibling = null;
if (rightSiblingPageId > 0) {
rightSibling = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightSiblingPageId),
false);
}
- // Lock is released in unsetSmPages(), after sm has fully completed.
- treeLatch.writeLock().lock();
try {
int rightPageId = freePageManager.getFreePage(ctx.metaFrame);
ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId),
true);
rightNode.acquireWriteLatch();
try {
- IBTreeLeafFrame rightFrame = ctx.createLeafFrame();
+ IBTreeLeafFrame rightFrame = ctx.createLeafFrame();
rightFrame.setPage(rightNode);
rightFrame.initBuffer((byte) 0);
rightFrame.setMultiComparator(cmp);
@@ -385,7 +383,7 @@
ctx.splitKey.setPages(pageId, rightPageId);
if (rightSibling != null) {
- rightSibling.acquireWriteLatch();
+ rightSibling.acquireWriteLatch();
try {
// Reuse rightFrame for modification.
rightFrame.setPage(rightSibling);
@@ -406,12 +404,14 @@
bufferCache.unpin(rightSibling);
}
}
+ return false;
}
- private void updateLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+ private boolean updateLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
ctx.leafFrame.setPage(node);
int oldTupleIndex = ctx.leafFrame.findUpdateTupleIndex(tuple);
FrameOpSpaceStatus spaceStatus = ctx.leafFrame.hasSpaceUpdate(tuple, oldTupleIndex);
+ boolean restartOp = false;
switch (spaceStatus) {
case SUFFICIENT_INPLACE_SPACE: {
ctx.leafFrame.update(tuple, oldTupleIndex, true);
@@ -443,13 +443,14 @@
ctx.leafFrame.insert(tuple, targetTupleIndex);
ctx.splitKey.reset();
} else {
- performLeafSplit(pageId, tuple, ctx);
+ restartOp = performLeafSplit(pageId, tuple, ctx);
}
break;
}
}
node.releaseWriteLatch();
bufferCache.unpin(node);
+ return restartOp;
}
private void insertInterior(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx)
@@ -506,95 +507,102 @@
}
}
- private void deleteLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+ private boolean deleteLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
ctx.leafFrame.setPage(node);
int tupleIndex = ctx.leafFrame.findDeleteTupleIndex(tuple);
-
+
// Will this leaf become empty?
if (ctx.leafFrame.getTupleCount() > 1) {
// Leaf will not become empty.
ctx.leafFrame.delete(tuple, tupleIndex);
node.releaseWriteLatch();
bufferCache.unpin(node);
- return;
+ return false;
}
// Leaf will become empty.
- IBTreeLeafFrame siblingFrame = ctx.createLeafFrame();
+ IBTreeLeafFrame siblingFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
siblingFrame.setMultiComparator(cmp);
ICachedPage leftNode = null;
ICachedPage rightNode = null;
int nextLeaf = ctx.leafFrame.getNextLeaf();
int prevLeaf = ctx.leafFrame.getPrevLeaf();
- if (prevLeaf > 0) {
- leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, prevLeaf), false);
+ // Try to get the tree latch, if it's already taken, then restart this operation
+ // to avoid latch deadlock.
+ if (!treeLatch.writeLock().tryLock()) {
+ return true;
}
try {
- if (nextLeaf > 0) {
- rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeaf), false);
- }
- try {
- treeLatch.writeLock().lock();
- try {
- ctx.leafFrame.delete(tuple, tupleIndex);
- // To propagate the deletion we only need to make the
- // splitKey != null.
- // Reuse data to identify which key to delete in the parent.
- ctx.splitKey.initData(1);
- } catch (Exception e) {
- // Don't propagate deletion.
- ctx.splitKey.reset();
- throw e;
- }
+ if (prevLeaf > 0) {
+ leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, prevLeaf), false);
+ }
+ try {
+ if (nextLeaf > 0) {
+ rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeaf), false);
+ }
+ try {
+ try {
+ ctx.leafFrame.delete(tuple, tupleIndex);
+ // To propagate the deletion we only need to make the
+ // splitKey != null.
+ // Reuse data to identify which key to delete in the parent.
+ ctx.splitKey.initData(1);
+ } catch (Exception e) {
+ // Don't propagate deletion.
+ ctx.splitKey.reset();
+ throw e;
+ }
- // TODO: Tie together with logging.
- ctx.leafFrame.setPageLsn(ctx.leafFrame.getPageLsn() + 1);
- ctx.leafFrame.setLevel(freePageManager.getFreePageLevelIndicator());
+ // TODO: Tie together with logging.
+ ctx.leafFrame.setPageLsn(ctx.leafFrame.getPageLsn() + 1);
+ ctx.leafFrame.setLevel(freePageManager.getFreePageLevelIndicator());
- ctx.smPages.add(pageId);
- ctx.leafFrame.setSmFlag(true);
+ ctx.smPages.add(pageId);
+ ctx.leafFrame.setSmFlag(true);
- node.releaseWriteLatch();
- bufferCache.unpin(node);
+ node.releaseWriteLatch();
+ bufferCache.unpin(node);
- if (leftNode != null) {
- leftNode.acquireWriteLatch();
- try {
- siblingFrame.setPage(leftNode);
- siblingFrame.setNextLeaf(nextLeaf);
- // TODO: Tie together with logging.
- siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
- } finally {
- leftNode.releaseWriteLatch();
- }
- }
+ if (leftNode != null) {
+ leftNode.acquireWriteLatch();
+ try {
+ siblingFrame.setPage(leftNode);
+ siblingFrame.setNextLeaf(nextLeaf);
+ // TODO: Tie together with logging.
+ siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
+ } finally {
+ leftNode.releaseWriteLatch();
+ }
+ }
- if (rightNode != null) {
- rightNode.acquireWriteLatch();
- try {
- siblingFrame.setPage(rightNode);
- siblingFrame.setPrevLeaf(prevLeaf);
- // TODO: Tie together with logging.
- siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
- } finally {
- rightNode.releaseWriteLatch();
- }
- }
- // Register pageId as a free.
- ctx.freePages.add(pageId);
- } catch (Exception e) {
- treeLatch.writeLock().unlock();
- throw e;
- } finally {
- if (rightNode != null) {
- bufferCache.unpin(rightNode);
- }
- }
- } finally {
- if (leftNode != null) {
- bufferCache.unpin(leftNode);
- }
+ if (rightNode != null) {
+ rightNode.acquireWriteLatch();
+ try {
+ siblingFrame.setPage(rightNode);
+ siblingFrame.setPrevLeaf(prevLeaf);
+ // TODO: Tie together with logging.
+ siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
+ } finally {
+ rightNode.releaseWriteLatch();
+ }
+ }
+ // Register pageId as a free.
+ ctx.freePages.add(pageId);
+ } finally {
+ if (rightNode != null) {
+ bufferCache.unpin(rightNode);
+ }
+ }
+ } finally {
+ if (leftNode != null) {
+ bufferCache.unpin(leftNode);
+ }
+ }
+ } catch (Exception e) {
+ treeLatch.writeLock().unlock();
+ throw e;
}
+ return false;
}
private void deleteInterior(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx)
@@ -665,7 +673,7 @@
// this check performs an unprotected read in the page
// the following could happen: TODO fill out
boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
- acquireLatch(node, ctx, unsafeIsLeaf);
+ acquireLatch(node, ctx, unsafeIsLeaf);
boolean smFlag = ctx.interiorFrame.getSmFlag();
// re-check leafness after latching
boolean isLeaf = ctx.interiorFrame.isLeaf();
@@ -674,7 +682,6 @@
// structure modification
ctx.pageLsns.add(ctx.interiorFrame.getPageLsn());
try {
-
// latch coupling, note: parent should never be write latched,
// otherwise something is wrong.
if (parent != null) {
@@ -742,8 +749,6 @@
} // end while
} else { // smFlag
ctx.opRestarts++;
- System.out.println("ONGOING SM ON PAGE " + pageId + " AT LEVEL " + ctx.interiorFrame.getLevel()
- + ", RESTARTS: " + ctx.opRestarts);
releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
@@ -752,9 +757,9 @@
// instead we just immediately release the lock. this is
// inefficient but still correct and will not cause
// latch-deadlock
- treeLatch.readLock().lock();
- treeLatch.readLock().unlock();
-
+ treeLatch.writeLock().lock();
+ treeLatch.writeLock().unlock();
+
// unwind recursion and restart operation, find lowest page
// with a pageLsn as seen by this operation during descent
ctx.pageLsns.removeLast(); // pop current page lsn
@@ -763,17 +768,19 @@
ctx.pageLsns.add(RESTART_OP);
}
} else { // isLeaf and !smFlag
- switch (ctx.op) {
+ // We may have to restart an op to avoid latch deadlock.
+ boolean restartOp = false;
+ switch (ctx.op) {
case INSERT: {
- insertLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+ restartOp = insertLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
break;
}
case UPDATE: {
- updateLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+ restartOp = updateLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
break;
}
case DELETE: {
- deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+ restartOp = deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
break;
}
case SEARCH: {
@@ -782,22 +789,27 @@
break;
}
}
+ if (restartOp) {
+ ctx.pageLsns.removeLast();
+ ctx.pageLsns.add(RESTART_OP);
+ }
}
} catch (TreeIndexException e) {
- if (!ctx.exceptionHandled) {
+ if (!ctx.exceptionHandled) {
releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
ctx.exceptionHandled = true;
}
throw e;
} catch (PageAllocationException e) {
- if (!ctx.exceptionHandled) {
+ if (!ctx.exceptionHandled) {
releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
ctx.exceptionHandled = true;
}
throw e;
} catch (Exception e) {
+ e.printStackTrace();
releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
BTreeException wrappedException = new BTreeException(e);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
index 621281a..80ac173 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
@@ -15,14 +15,14 @@
package edu.uci.ics.hyracks.storage.am.btree.impls;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
public class FieldPrefixPrefixTupleReference extends TypeAwareTupleReference {
- public FieldPrefixPrefixTupleReference(ITypeTrait[] typeTraits) {
+ public FieldPrefixPrefixTupleReference(ITypeTraits[] typeTraits) {
super(typeTraits);
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
index 625fa02..f6310af 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
@@ -1,7 +1,7 @@
package edu.uci.ics.hyracks.storage.am.btree.util;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrameFactory;
@@ -20,7 +20,7 @@
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
public class BTreeUtils {
- public static BTree createBTree(IBufferCache bufferCache, int btreeFileId, ITypeTrait[] typeTraits, IBinaryComparator[] cmps, BTreeLeafFrameType leafType) throws BTreeException {
+ public static BTree createBTree(IBufferCache bufferCache, int btreeFileId, ITypeTraits[] typeTraits, IBinaryComparator[] cmps, BTreeLeafFrameType leafType) throws BTreeException {
MultiComparator cmp = new MultiComparator(cmps);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index 459ab57..ff4d078 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -40,7 +40,7 @@
protected final IStorageManagerInterface storageManager;
protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
- protected final ITypeTrait[] typeTraits;
+ protected final ITypeTraits[] typeTraits;
protected final IIndexDataflowHelperFactory dataflowHelperFactory;
public AbstractTreeIndexOperatorDescriptor(JobSpecification spec,
@@ -49,7 +49,7 @@
IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories,
IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, inputArity, outputArity);
@@ -77,7 +77,7 @@
}
@Override
- public ITypeTrait[] getTreeIndexTypeTraits() {
+ public ITypeTraits[] getTreeIndexTypeTraits() {
return typeTraits;
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
index 9bbca56..e415cbb 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
public interface ITreeIndexOperatorDescriptor extends IIndexOperatorDescriptor {
@@ -26,6 +26,6 @@
public ITreeIndexFrameFactory getTreeIndexLeafFactory();
- public ITypeTrait[] getTreeIndexTypeTraits();
+ public ITypeTraits[] getTreeIndexTypeTraits();
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index c5b5355..af30105 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -35,7 +35,7 @@
public TreeIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
- ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
float fillFactor, IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, 1, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 8cc83f7..8d56d37 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -35,7 +35,7 @@
IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider,
fileSplitProvider, interiorFrameFactory, leafFrameFactory,
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 50fb3ea..e08f5d6 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -41,7 +41,7 @@
IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories,
int[] fieldPermutation, IndexOp op,
IIndexDataflowHelperFactory dataflowHelperFactory) {
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 96a2a94..4d718a8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -36,7 +36,7 @@
public TreeIndexStatsOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
- ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
+ ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
index 72a36a1..4776bdd 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
@@ -17,7 +17,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
@@ -29,11 +29,11 @@
protected int nullFlagsBytes;
protected int dataStartOff;
- protected ITypeTrait[] typeTraits;
+ protected ITypeTraits[] typeTraits;
protected VarLenIntEncoderDecoder encDec = new VarLenIntEncoderDecoder();
protected int[] decodedFieldSlots;
- public TypeAwareTupleReference(ITypeTrait[] typeTraits) {
+ public TypeAwareTupleReference(ITypeTraits[] typeTraits) {
this.typeTraits = typeTraits;
this.fieldStartIndex = 0;
setFieldCount(typeTraits.length);
@@ -50,12 +50,11 @@
int end = fieldStartIndex + fieldCount;
encDec.reset(buf.array(), tupleStartOff + nullFlagsBytes);
for (int i = fieldStartIndex; i < end; i++) {
- int staticDataLen = typeTraits[i].getStaticallyKnownDataLength();
- if (staticDataLen == ITypeTrait.VARIABLE_LENGTH) {
+ if (!typeTraits[i].isFixedLength()) {
cumul += encDec.decode();
decodedFieldSlots[field++] = cumul;
} else {
- cumul += staticDataLen;
+ cumul += typeTraits[i].getFixedLength();
decodedFieldSlots[field++] = cumul;
}
}
@@ -119,8 +118,8 @@
return (int) Math.ceil(fieldCount / 8.0);
}
- @Override
- public int getTupleSize() {
- return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount-1];
- }
+ @Override
+ public int getTupleSize() {
+ return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount - 1];
+ }
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
index 6b33712..fe52608 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
@@ -17,17 +17,17 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
public class TypeAwareTupleWriter implements ITreeIndexTupleWriter {
- protected ITypeTrait[] typeTraits;
+ protected ITypeTraits[] typeTraits;
protected VarLenIntEncoderDecoder encDec = new VarLenIntEncoderDecoder();
- public TypeAwareTupleWriter(ITypeTrait[] typeTraits) {
+ public TypeAwareTupleWriter(ITypeTraits[] typeTraits) {
this.typeTraits = typeTraits;
}
@@ -65,13 +65,13 @@
int nullFlagsBytes = getNullFlagsBytes(tuple);
// write null indicator bits
for (int i = 0; i < nullFlagsBytes; i++) {
- targetBuf[runner++] = (byte) 0;
+ targetBuf[runner++] = (byte) 0;
}
// write field slots for variable length fields
encDec.reset(targetBuf, runner);
for (int i = 0; i < tuple.getFieldCount(); i++) {
- if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+ if (!typeTraits[i].isFixedLength()) {
encDec.encode(tuple.getFieldLength(i));
}
}
@@ -81,14 +81,13 @@
for (int i = 0; i < tuple.getFieldCount(); i++) {
int s = tuple.getFieldStart(i);
int l = tuple.getFieldLength(i);
- System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner,
- tuple.getFieldLength(i));
+ System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
runner += tuple.getFieldLength(i);
}
return runner - targetOff;
}
-
+
@Override
public int writeTupleFields(ITupleReference tuple, int startField, int numFields, ByteBuffer targetBuf,
int targetOff) {
@@ -102,7 +101,7 @@
// write field slots for variable length fields
encDec.reset(targetBuf.array(), runner);
for (int i = startField; i < startField + numFields; i++) {
- if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+ if (!typeTraits[i].isFixedLength()) {
encDec.encode(tuple.getFieldLength(i));
}
}
@@ -124,7 +123,7 @@
protected int getFieldSlotsBytes(ITupleReference tuple) {
int fieldSlotBytes = 0;
for (int i = 0; i < tuple.getFieldCount(); i++) {
- if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+ if (!typeTraits[i].isFixedLength()) {
fieldSlotBytes += encDec.getBytesRequired(tuple.getFieldLength(i));
}
}
@@ -138,18 +137,18 @@
protected int getFieldSlotsBytes(ITupleReference tuple, int startField, int numFields) {
int fieldSlotBytes = 0;
for (int i = startField; i < startField + numFields; i++) {
- if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+ if (!typeTraits[i].isFixedLength()) {
fieldSlotBytes += encDec.getBytesRequired(tuple.getFieldLength(i));
}
}
return fieldSlotBytes;
}
- public ITypeTrait[] getTypeTraits() {
+ public ITypeTraits[] getTypeTraits() {
return typeTraits;
}
- public void setTypeTraits(ITypeTrait[] typeTraits) {
+ public void setTypeTraits(ITypeTraits[] typeTraits) {
this.typeTraits = typeTraits;
}
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
index 28e2535..9e6ba6f 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.tuples;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
@@ -23,9 +23,9 @@
ITreeIndexTupleWriterFactory {
private static final long serialVersionUID = 1L;
- private ITypeTrait[] typeTraits;
+ private ITypeTraits[] typeTraits;
- public TypeAwareTupleWriterFactory(ITypeTrait[] typeTraits) {
+ public TypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
this.typeTraits = typeTraits;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
index 499b60b..1dc753e 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
@@ -16,14 +16,14 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.api;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptor;
public interface IInvertedIndexOperatorDescriptor extends ITreeIndexOperatorDescriptor {
public IBinaryComparatorFactory[] getInvListsComparatorFactories();
- public ITypeTrait[] getInvListsTypeTraits();
+ public ITypeTraits[] getInvListsTypeTraits();
public IFileSplitProvider getInvListsFileSplitProvider();
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
index a8860e2..eab266d 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -45,21 +45,21 @@
// Btree.
protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
- protected final ITypeTrait[] btreeTypeTraits;
+ protected final ITypeTraits[] btreeTypeTraits;
protected final IBinaryComparatorFactory[] btreeComparatorFactories;
protected final IIndexDataflowHelperFactory btreeDataflowHelperFactory;
protected final IFileSplitProvider btreeFileSplitProvider;
// Inverted index.
- protected final ITypeTrait[] invListsTypeTraits;
+ protected final ITypeTraits[] invListsTypeTraits;
protected final IBinaryComparatorFactory[] invListComparatorFactories;
protected final IFileSplitProvider invListsFileSplitProvider;
public AbstractInvertedIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
RecordDescriptor recDesc, IStorageManagerInterface storageManager,
IFileSplitProvider btreeFileSplitProvider, IFileSplitProvider invListsFileSplitProvider,
- IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTrait[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTrait[] invListsTypeTraits,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
IBinaryComparatorFactory[] invListComparatorFactories,
IIndexDataflowHelperFactory btreeDataflowHelperFactory) {
super(spec, inputArity, outputArity);
@@ -103,7 +103,7 @@
}
@Override
- public ITypeTrait[] getTreeIndexTypeTraits() {
+ public ITypeTraits[] getTreeIndexTypeTraits() {
return btreeTypeTraits;
}
@@ -133,7 +133,7 @@
}
@Override
- public ITypeTrait[] getInvListsTypeTraits() {
+ public ITypeTraits[] getInvListsTypeTraits() {
return invListsTypeTraits;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 0af27d6..d00bea6 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -59,7 +59,7 @@
public void open() throws HyracksDataException {
accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
writeBuffer = ctx.allocateFrame();
- builder = new ArrayTupleBuilder(outputRecDesc.getFields().length);
+ builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
builderDos = builder.getDataOutput();
appender = new FrameTupleAppender(ctx.getFrameSize());
appender.reset(writeBuffer, true);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index 05f8b7d..e91aa08 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -36,8 +36,8 @@
public InvertedIndexBulkLoadOperatorDescriptor(JobSpecification spec, int[] fieldPermutation,
IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
- ITypeTrait[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
- ITypeTrait[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+ ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+ ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
IIndexDataflowHelperFactory btreeDataflowHelperFactory) {
super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
index f1704cc..bf9899f 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -43,8 +43,8 @@
public InvertedIndexSearchOperatorDescriptor(JobSpecification spec,
int queryField, IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
- ITypeTrait[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
- ITypeTrait[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+ ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+ ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc) {
super(spec, 1, 1, recDesc, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
index 03fc9a1..643c105 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -26,10 +26,10 @@
private byte[] targetBuf;
private int pos;
- public FixedSizeElementInvertedListBuilder(ITypeTrait[] invListFields) {
+ public FixedSizeElementInvertedListBuilder(ITypeTraits[] invListFields) {
int tmp = 0;
for (int i = 0; i < invListFields.length; i++) {
- tmp += invListFields[i].getStaticallyKnownDataLength();
+ tmp += invListFields[i].getFixedLength();
}
listElementSize = tmp;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
index f7ef56e..4158019 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
@@ -5,7 +5,7 @@
import java.io.DataInputStream;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -32,7 +32,7 @@
private ICachedPage[] pages = new ICachedPage[10];
private int[] elementIndexes = new int[10];
- public FixedSizeElementInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTrait[] invListFields) {
+ public FixedSizeElementInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields) {
this.bufferCache = bufferCache;
this.fileId = fileId;
this.currentElementIx = 0;
@@ -40,7 +40,7 @@
int tmp = 0;
for (int i = 0; i < invListFields.length; i++) {
- tmp += invListFields[i].getStaticallyKnownDataLength();
+ tmp += invListFields[i].getFixedLength();
}
elementSize = tmp;
this.currentOff = -elementSize;
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
index 9858eb0..cbedc45 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
@@ -19,29 +19,29 @@
import edu.uci.ics.hyracks.api.comm.FrameHelper;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
public class FixedSizeFrameTupleAccessor implements IFrameTupleAccessor {
private final int frameSize;
private ByteBuffer buffer;
- private final ITypeTrait[] fields;
+ private final ITypeTraits[] fields;
private final int[] fieldStartOffsets;
private final int tupleSize;
- public FixedSizeFrameTupleAccessor(int frameSize, ITypeTrait[] fields) {
+ public FixedSizeFrameTupleAccessor(int frameSize, ITypeTraits[] fields) {
this.frameSize = frameSize;
this.fields = fields;
this.fieldStartOffsets = new int[fields.length];
this.fieldStartOffsets[0] = 0;
for (int i = 1; i < fields.length; i++) {
- fieldStartOffsets[i] = fieldStartOffsets[i - 1] + fields[i - 1].getStaticallyKnownDataLength();
+ fieldStartOffsets[i] = fieldStartOffsets[i - 1] + fields[i - 1].getFixedLength();
}
int tmp = 0;
for (int i = 0; i < fields.length; i++) {
- tmp += fields[i].getStaticallyKnownDataLength();
+ tmp += fields[i].getFixedLength();
}
tupleSize = tmp;
}
@@ -58,12 +58,12 @@
@Override
public int getFieldEndOffset(int tupleIndex, int fIdx) {
- return getTupleStartOffset(tupleIndex) + fieldStartOffsets[fIdx] + fields[fIdx].getStaticallyKnownDataLength();
+ return getTupleStartOffset(tupleIndex) + fieldStartOffsets[fIdx] + fields[fIdx].getFixedLength();
}
@Override
public int getFieldLength(int tupleIndex, int fIdx) {
- return fields[fIdx].getStaticallyKnownDataLength();
+ return fields[fIdx].getFixedLength();
}
@Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
index edc2304..489ec2e 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
@@ -18,7 +18,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
public class FixedSizeFrameTupleAppender {
@@ -29,11 +29,11 @@
private int tupleCount;
private int tupleDataEndOffset;
- public FixedSizeFrameTupleAppender(int frameSize, ITypeTrait[] fields) {
+ public FixedSizeFrameTupleAppender(int frameSize, ITypeTraits[] fields) {
this.frameSize = frameSize;
int tmp = 0;
for (int i = 0; i < fields.length; i++) {
- tmp += fields[i].getStaticallyKnownDataLength();
+ tmp += fields[i].getFixedLength();
}
tupleSize = tmp;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
index 248b81e..0656d69 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
@@ -15,22 +15,22 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
public class FixedSizeTupleReference implements ITupleReference {
- private final ITypeTrait[] typeTraits;
+ private final ITypeTraits[] typeTraits;
private final int[] fieldStartOffsets;
private byte[] data;
private int startOff;
- public FixedSizeTupleReference(ITypeTrait[] typeTraits) {
+ public FixedSizeTupleReference(ITypeTraits[] typeTraits) {
this.typeTraits = typeTraits;
this.fieldStartOffsets = new int[typeTraits.length];
this.fieldStartOffsets[0] = 0;
for (int i = 1; i < typeTraits.length; i++) {
- fieldStartOffsets[i] = fieldStartOffsets[i - 1] + typeTraits[i - 1].getStaticallyKnownDataLength();
+ fieldStartOffsets[i] = fieldStartOffsets[i - 1] + typeTraits[i - 1].getFixedLength();
}
}
@@ -51,7 +51,7 @@
@Override
public int getFieldLength(int fIdx) {
- return typeTraits[fIdx].getStaticallyKnownDataLength();
+ return typeTraits[fIdx].getFixedLength();
}
@Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 1118d04..986e57b 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -18,7 +18,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -57,12 +57,12 @@
private int rootPageId = 0;
private IBufferCache bufferCache;
private int fileId;
- private final ITypeTrait[] invListTypeTraits;
+ private final ITypeTraits[] invListTypeTraits;
private final MultiComparator invListCmp;
private final int numTokenFields;
private final int numInvListKeys;
- public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTrait[] invListTypeTraits, MultiComparator invListCmp) {
+ public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTraits[] invListTypeTraits, MultiComparator invListCmp) {
this.bufferCache = bufferCache;
this.btree = btree;
this.invListCmp = invListCmp;
@@ -230,7 +230,7 @@
return invListCmp;
}
- public ITypeTrait[] getTypeTraits() {
+ public ITypeTraits[] getTypeTraits() {
return invListTypeTraits;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index dd09ef7..b24d416 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -24,10 +24,10 @@
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -72,14 +72,14 @@
protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
- protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFields().length);
+ protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFieldCount());
protected DataOutput queryTokenDos = queryTokenBuilder.getDataOutput();
protected FrameTupleAppender queryTokenAppender;
protected ByteBuffer queryTokenFrame;
protected final InvertedIndex invIndex;
protected final IBinaryTokenizer queryTokenizer;
- protected final ITypeTrait[] invListFieldsWithCount;
+ protected final ITypeTraits[] invListFieldsWithCount;
protected int occurrenceThreshold;
protected final int cursorCacheSize = 10;
@@ -95,15 +95,15 @@
interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
- ITypeTrait[] invListFields = invIndex.getTypeTraits();
- invListFieldsWithCount = new TypeTrait[invListFields.length + 1];
+ ITypeTraits[] invListFields = invIndex.getTypeTraits();
+ invListFieldsWithCount = new ITypeTraits[invListFields.length + 1];
int tmp = 0;
for (int i = 0; i < invListFields.length; i++) {
invListFieldsWithCount[i] = invListFields[i];
- tmp += invListFields[i].getStaticallyKnownDataLength();
+ tmp += invListFields[i].getFixedLength();
}
// using an integer for counting occurrences
- invListFieldsWithCount[invListFields.length] = new TypeTrait(4);
+ invListFieldsWithCount[invListFields.length] = IntegerPointable.TYPE_TRAITS;
invListKeyLength = tmp;
resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
@@ -126,7 +126,7 @@
queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
queryTokenFrame = ctx.allocateFrame();
-
+
btreeAccessor = invIndex.getBTree().createAccessor();
currentNumResults = 0;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
index bbb32d6..be6bfb0 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
@@ -19,60 +19,59 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-public abstract class AbstractUTF8StringBinaryTokenizer implements
- IBinaryTokenizer {
+public abstract class AbstractUTF8StringBinaryTokenizer implements IBinaryTokenizer {
- protected byte[] data;
- protected int start;
- protected int length;
- protected int tokenLength;
- protected int index;
- protected int utf8Length;
+ protected byte[] data;
+ protected int start;
+ protected int length;
+ protected int tokenLength;
+ protected int index;
+ protected int utf8Length;
- protected final IntArray tokensStart;
- protected final IntArray tokensLength;
- protected final IToken token;
+ protected final IntArray tokensStart;
+ protected final IntArray tokensLength;
+ protected final IToken token;
- protected final boolean ignoreTokenCount;
- protected final boolean sourceHasTypeTag;
+ protected final boolean ignoreTokenCount;
+ protected final boolean sourceHasTypeTag;
- public AbstractUTF8StringBinaryTokenizer(boolean ignoreTokenCount,
- boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
- this.ignoreTokenCount = ignoreTokenCount;
- this.sourceHasTypeTag = sourceHasTypeTag;
- if (!ignoreTokenCount) {
- tokensStart = new IntArray();
- tokensLength = new IntArray();
- } else {
- tokensStart = null;
- tokensLength = null;
- }
- token = tokenFactory.createToken();
- }
+ public AbstractUTF8StringBinaryTokenizer(boolean ignoreTokenCount, boolean sourceHasTypeTag,
+ ITokenFactory tokenFactory) {
+ this.ignoreTokenCount = ignoreTokenCount;
+ this.sourceHasTypeTag = sourceHasTypeTag;
+ if (!ignoreTokenCount) {
+ tokensStart = new IntArray();
+ tokensLength = new IntArray();
+ } else {
+ tokensStart = null;
+ tokensLength = null;
+ }
+ token = tokenFactory.createToken();
+ }
- @Override
- public IToken getToken() {
- return token;
- }
+ @Override
+ public IToken getToken() {
+ return token;
+ }
- @Override
- public void reset(byte[] data, int start, int length) {
- this.start = start;
- index = this.start;
- if (sourceHasTypeTag) {
- index++; // skip type tag
- }
- utf8Length = StringUtils.getUTFLen(data, index);
- index += 2; // skip utf8 length indicator
- this.data = data;
- this.length = length + start;
+ @Override
+ public void reset(byte[] data, int start, int length) {
+ this.start = start;
+ index = this.start;
+ if (sourceHasTypeTag) {
+ index++; // skip type tag
+ }
+ utf8Length = UTF8StringPointable.getUTFLen(data, index);
+ index += 2; // skip utf8 length indicator
+ this.data = data;
+ this.length = length + start;
- tokenLength = 0;
- if (!ignoreTokenCount) {
- tokensStart.reset();
- tokensLength.reset();
- }
- }
+ tokenLength = 0;
+ if (!ignoreTokenCount) {
+ tokensStart.reset();
+ tokensLength.reset();
+ }
+ }
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
index 209c939..65afa65 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
@@ -21,7 +21,7 @@
import java.io.DataOutput;
import java.io.IOException;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
public abstract class AbstractUTF8Token implements IToken {
public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
@@ -58,9 +58,9 @@
int lowerCaseUTF8Len = 0;
int pos = start;
for (int i = 0; i < size; i++) {
- char c = Character.toLowerCase(StringUtils.charAt(data, pos));
- lowerCaseUTF8Len += StringUtils.getModifiedUTF8Len(c);
- pos += StringUtils.charSize(data, pos);
+ char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+ lowerCaseUTF8Len += UTF8StringPointable.getModifiedUTF8Len(c);
+ pos += UTF8StringPointable.charSize(data, pos);
}
return lowerCaseUTF8Len;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
index 83d0c75..9dacde6 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
@@ -19,7 +19,7 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
public class DelimitedUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
@@ -31,8 +31,8 @@
@Override
public boolean hasNext() {
// skip delimiters
- while (index < length && isSeparator(StringUtils.charAt(data, index))) {
- index += StringUtils.charSize(data, index);
+ while (index < length && isSeparator(UTF8StringPointable.charAt(data, index))) {
+ index += UTF8StringPointable.charSize(data, index);
}
return index < length;
}
@@ -45,8 +45,8 @@
public void next() {
tokenLength = 0;
int currentTokenStart = index;
- while (index < length && !isSeparator(StringUtils.charAt(data, index))) {
- index += StringUtils.charSize(data, index);
+ while (index < length && !isSeparator(UTF8StringPointable.charAt(data, index))) {
+ index += UTF8StringPointable.charSize(data, index);
tokenLength++;
}
int tokenCount = 1;
@@ -60,12 +60,12 @@
int currLength = 0;
while (currLength < tokenLength) {
// case insensitive comparison
- if (Character.toLowerCase(StringUtils.charAt(data, currentTokenStart + offset)) != Character
- .toLowerCase(StringUtils.charAt(data, tokenStart + offset))) {
+ if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
+ .toLowerCase(UTF8StringPointable.charAt(data, tokenStart + offset))) {
tokenCount--;
break;
}
- offset += StringUtils.charSize(data, currentTokenStart + offset);
+ offset += UTF8StringPointable.charSize(data, currentTokenStart + offset);
currLength++;
}
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
index 43f89c7..b7bb828 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
@@ -22,7 +22,7 @@
import java.io.DataOutput;
import java.io.IOException;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
public class HashedUTF8NGramToken extends UTF8NGramToken {
public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
@@ -45,9 +45,9 @@
int numRegGrams = tokenLength - numPreChars - numPostChars;
int pos = start;
for (int i = 0; i < numRegGrams; i++) {
- hash ^= Character.toLowerCase(StringUtils.charAt(data, pos));
+ hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
hash *= GOLDEN_RATIO_32;
- pos += StringUtils.charSize(data, pos);
+ pos += UTF8StringPointable.charSize(data, pos);
}
// post chars
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
index 747b65d..42ed053 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
@@ -22,7 +22,7 @@
import java.io.DataOutput;
import java.io.IOException;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
public class HashedUTF8WordToken extends UTF8WordToken {
@@ -46,10 +46,11 @@
}
int offset = 0;
for (int i = 0; i < tokenLength; i++) {
- if (StringUtils.charAt(t.getData(), t.getStart() + offset) != StringUtils.charAt(data, start + offset)) {
+ if (UTF8StringPointable.charAt(t.getData(), t.getStart() + offset) != UTF8StringPointable.charAt(data,
+ start + offset)) {
return false;
}
- offset += StringUtils.charSize(data, start + offset);
+ offset += UTF8StringPointable.charSize(data, start + offset);
}
return true;
}
@@ -67,9 +68,9 @@
int pos = start;
hash = GOLDEN_RATIO_32;
for (int i = 0; i < tokenLength; i++) {
- hash ^= Character.toLowerCase(StringUtils.charAt(data, pos));
+ hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
hash *= GOLDEN_RATIO_32;
- pos += StringUtils.charSize(data, pos);
+ pos += UTF8StringPointable.charSize(data, pos);
}
hash += tokenCount;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
index bdbf6f8..fdfc02f 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
@@ -19,7 +19,7 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
public class NGramUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
@@ -62,7 +62,7 @@
concreteToken.setNumPrePostChars(numPreChars, numPostChars);
if (numPreChars == 0) {
- index += StringUtils.charSize(data, index);
+ index += UTF8StringPointable.charSize(data, index);
}
// compute token count
@@ -73,14 +73,14 @@
tokenCount++; // assume found
int offset = 0;
for (int j = 0; j < gramLength; j++) {
- if (Character.toLowerCase(StringUtils.charAt(data, currentTokenStart + offset)) != Character
- .toLowerCase(StringUtils.charAt(data, tmpIndex + offset))) {
+ if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
+ .toLowerCase(UTF8StringPointable.charAt(data, tmpIndex + offset))) {
tokenCount--;
break;
}
- offset += StringUtils.charSize(data, tmpIndex + offset);
+ offset += UTF8StringPointable.charSize(data, tmpIndex + offset);
}
- tmpIndex += StringUtils.charSize(data, tmpIndex);
+ tmpIndex += UTF8StringPointable.charSize(data, tmpIndex);
}
}
@@ -98,7 +98,7 @@
int end = pos + utf8Length;
while (pos < end) {
numChars++;
- pos += StringUtils.charSize(data, pos);
+ pos += UTF8StringPointable.charSize(data, pos);
}
if (usePrePost) {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
index 1b124dc..59cadc8 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
@@ -22,6 +22,7 @@
import java.io.DataOutput;
import java.io.IOException;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
@@ -67,9 +68,9 @@
int pos = start;
for (int i = 0; i < numRegChars; i++) {
- char c = Character.toLowerCase(StringUtils.charAt(data, pos));
+ char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
StringUtils.writeCharAsModifiedUTF8(c, dos);
- pos += StringUtils.charSize(data, pos);
+ pos += UTF8StringPointable.charSize(data, pos);
}
// post chars
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
index 2a74145..97a1e12 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
@@ -22,6 +22,7 @@
import java.io.DataOutput;
import java.io.IOException;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
public class UTF8WordToken extends AbstractUTF8Token {
@@ -38,9 +39,9 @@
StringUtils.writeUTF8Len(tokenUTF8Len, dos);
int pos = start;
for (int i = 0; i < tokenLength; i++) {
- char c = Character.toLowerCase(StringUtils.charAt(data, pos));
+ char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
StringUtils.writeCharAsModifiedUTF8(c, dos);
- pos += StringUtils.charSize(data, pos);
+ pos += UTF8StringPointable.charSize(data, pos);
}
}
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
index 5a62f51..a1d1f06 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
@@ -15,33 +15,33 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.util;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
public class InvertedIndexUtils {
- // Type traits to be appended to the token type trait which finally form the BTree field type traits.
- private static final ITypeTrait[] btreeValueTypeTraits = new ITypeTrait[4];
- static {
- // startPageId
- btreeValueTypeTraits[0] = new TypeTrait(4);
+ // Type traits to be appended to the token type trait which finally form the BTree field type traits.
+ private static final ITypeTraits[] btreeValueTypeTraits = new ITypeTraits[4];
+ static {
+ // startPageId
+ btreeValueTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
// endPageId
- btreeValueTypeTraits[1] = new TypeTrait(4);
+ btreeValueTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
// startOff
- btreeValueTypeTraits[2] = new TypeTrait(4);
+ btreeValueTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
// numElements
- btreeValueTypeTraits[3] = new TypeTrait(4);
- }
-
- public static ITypeTrait[] getBTreeTypeTraits(ITypeTrait[] tokenTypeTraits) {
- ITypeTrait[] btreeTypeTraits = new ITypeTrait[tokenTypeTraits.length + btreeValueTypeTraits.length];
- // Set key type traits.
- for (int i = 0; i < tokenTypeTraits.length; i++) {
- btreeTypeTraits[i] = tokenTypeTraits[i];
- }
- // Set value type traits.
- for (int i = 0; i < btreeValueTypeTraits.length; i++) {
- btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
- }
- return btreeTypeTraits;
- }
+ btreeValueTypeTraits[3] = IntegerPointable.TYPE_TRAITS;
+ }
+
+ public static ITypeTraits[] getBTreeTypeTraits(ITypeTraits[] tokenTypeTraits) {
+ ITypeTraits[] btreeTypeTraits = new ITypeTraits[tokenTypeTraits.length + btreeValueTypeTraits.length];
+ // Set key type traits.
+ for (int i = 0; i < tokenTypeTraits.length; i++) {
+ btreeTypeTraits[i] = tokenTypeTraits[i];
+ }
+ // Set value type traits.
+ for (int i = 0; i < btreeValueTypeTraits.length; i++) {
+ btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
+ }
+ return btreeTypeTraits;
+ }
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 586324f..8cb4e55 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -19,7 +19,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -39,7 +39,7 @@
public RTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
index 96d739c..f3e6be2 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
@@ -19,8 +19,9 @@
import java.util.Collections;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
@@ -42,9 +43,9 @@
public class RTreeNSMInteriorFrame extends RTreeNSMFrame implements IRTreeInteriorFrame {
private static final int childPtrSize = 4;
- private static IBinaryComparator childPtrCmp = IntegerBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
-
+ private static IBinaryComparator childPtrCmp = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY)
+ .createBinaryComparator();
+
public RTreeNSMInteriorFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders) {
super(tupleWriter, keyValueProviders);
frameTuple.setFieldCount(keyValueProviders.length);
@@ -269,11 +270,11 @@
}
private int pointerCmp(ITupleReference tupleA, ITupleReference tupleB, MultiComparator cmp) {
- return childPtrCmp.compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1),
- getChildPointerOff(tupleA), childPtrSize, tupleB.getFieldData(cmp.getKeyFieldCount() - 1),
- getChildPointerOff(tupleB), childPtrSize);
+ return childPtrCmp
+ .compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleA), childPtrSize,
+ tupleB.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleB), childPtrSize);
}
-
+
@Override
public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException {
RTreeSplitKey rTreeSplitKey = ((RTreeSplitKey) splitKey);
@@ -294,10 +295,10 @@
for (int k = 0; k < getTupleCount(); ++k) {
frameTuple.resetByTupleIndex(this, k);
- double LowerKey = keyValueProviders[i].getValue(frameTuple.getFieldData(i),
- frameTuple.getFieldStart(i));
- double UpperKey = keyValueProviders[j].getValue(frameTuple.getFieldData(j),
- frameTuple.getFieldStart(j));
+ double LowerKey = keyValueProviders[i]
+ .getValue(frameTuple.getFieldData(i), frameTuple.getFieldStart(i));
+ double UpperKey = keyValueProviders[j]
+ .getValue(frameTuple.getFieldData(j), frameTuple.getFieldStart(j));
tupleEntries1.add(k, LowerKey);
tupleEntries2.add(k, UpperKey);
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
index 244c69e..898ebf8 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
@@ -17,18 +17,18 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
public class RTreeTypeAwareTupleWriter extends TypeAwareTupleWriter {
- public RTreeTypeAwareTupleWriter(ITypeTrait[] typeTraits) {
+ public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits) {
super(typeTraits);
}
- public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {
- int runner = targetOff;
+ public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {
+ int runner = targetOff;
int nullFlagsBytes = getNullFlagsBytes(refs.length);
// write null indicator bits
for (int i = 0; i < nullFlagsBytes; i++) {
@@ -39,7 +39,7 @@
// since the r-tree has fixed length keys, we don't actually need this?
encDec.reset(targetBuf.array(), runner);
for (int i = startField; i < startField + refs.length; i++) {
- if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+ if (!typeTraits[i].isFixedLength()) {
encDec.encode(refs[i].getFieldLength(i));
}
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
index a27d8cd..24b2b53 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
@@ -15,16 +15,16 @@
package edu.uci.ics.hyracks.storage.am.rtree.tuples;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
public class RTreeTypeAwareTupleWriterFactory implements ITreeIndexTupleWriterFactory {
private static final long serialVersionUID = 1L;
- private ITypeTrait[] typeTraits;
+ private ITypeTraits[] typeTraits;
- public RTreeTypeAwareTupleWriterFactory(ITypeTrait[] typeTraits) {
+ public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
this.typeTraits = typeTraits;
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
index b312116..74e4840 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
@@ -15,88 +15,16 @@
package edu.uci.ics.hyracks.storage.am.rtree.util;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.FloatBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.FloatBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.FloatPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.IntegerPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.data.PointablePrimitiveValueProviderFactory;
public class RTreeUtils {
- public static IPrimitiveValueProvider comparatorToPrimitiveValueProvider(IBinaryComparator cmp) {
- if (cmp instanceof IntegerBinaryComparator) {
- return IntegerPrimitiveValueProviderFactory.INSTANCE.createPrimitiveValueProvider();
+ public static IPrimitiveValueProviderFactory[] createPrimitiveValueProviderFactories(int len, IPointableFactory pf) {
+ IPrimitiveValueProviderFactory[] pvpfs = new IPrimitiveValueProviderFactory[len];
+ for (int i = 0; i < len; ++i) {
+ pvpfs[i] = new PointablePrimitiveValueProviderFactory(pf);
}
- if (cmp instanceof FloatBinaryComparator) {
- return FloatPrimitiveValueProviderFactory.INSTANCE.createPrimitiveValueProvider();
- }
- if (cmp instanceof DoubleBinaryComparator) {
- return DoublePrimitiveValueProviderFactory.INSTANCE.createPrimitiveValueProvider();
- }
- throw new UnsupportedOperationException(
- "Converting binary comparator to primitive value provider not implemented for: " + cmp.toString());
- }
-
- public static IPrimitiveValueProvider[] comparatorsToPrimitiveValueProviders(IBinaryComparator[] cmps) {
- IPrimitiveValueProvider[] primitiveValueProviders = new IPrimitiveValueProvider[cmps.length];
- for (int i = 0; i < cmps.length; i++) {
- primitiveValueProviders[i] = comparatorToPrimitiveValueProvider(cmps[i]);
- }
- return primitiveValueProviders;
- }
-
- public static IPrimitiveValueProviderFactory comparatorToPrimitiveValueProviderFactory(IBinaryComparator cmp) {
- if (cmp instanceof IntegerBinaryComparator) {
- return IntegerPrimitiveValueProviderFactory.INSTANCE;
- }
- if (cmp instanceof FloatBinaryComparator) {
- return FloatPrimitiveValueProviderFactory.INSTANCE;
- }
- if (cmp instanceof DoubleBinaryComparator) {
- return DoublePrimitiveValueProviderFactory.INSTANCE;
- }
- throw new UnsupportedOperationException(
- "Converting binary comparator to primitive value provider factory not implemented for: "
- + cmp.toString());
- }
-
- public static IPrimitiveValueProviderFactory[] comparatorsToPrimitiveValueProviderFactories(IBinaryComparator[] cmps) {
- IPrimitiveValueProviderFactory[] primitiveValueProviders = new IPrimitiveValueProviderFactory[cmps.length];
- for (int i = 0; i < cmps.length; i++) {
- primitiveValueProviders[i] = comparatorToPrimitiveValueProviderFactory(cmps[i]);
- }
- return primitiveValueProviders;
- }
-
- public static IPrimitiveValueProviderFactory comparatorFactoryToPrimitiveValueProviderFactory(
- IBinaryComparatorFactory cmpFactory) {
- if (cmpFactory instanceof IntegerBinaryComparatorFactory) {
- return IntegerPrimitiveValueProviderFactory.INSTANCE;
- }
- if (cmpFactory instanceof FloatBinaryComparatorFactory) {
- return FloatPrimitiveValueProviderFactory.INSTANCE;
- }
- if (cmpFactory instanceof DoubleBinaryComparatorFactory) {
- return DoublePrimitiveValueProviderFactory.INSTANCE;
- }
- throw new UnsupportedOperationException(
- "Converting binary comparator factory to primitive value provider factory not implemented for: "
- + cmpFactory.toString());
- }
-
- public static IPrimitiveValueProviderFactory[] comparatorFactoriesToPrimitiveValueProviderFactories(
- IBinaryComparatorFactory[] cmpFactories) {
- IPrimitiveValueProviderFactory[] primitiveValueProviders = new IPrimitiveValueProviderFactory[cmpFactories.length];
- for (int i = 0; i < cmpFactories.length; i++) {
- primitiveValueProviders[i] = comparatorFactoryToPrimitiveValueProviderFactory(cmpFactories[i]);
- }
- return primitiveValueProviders;
+ return pvpfs;
}
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
index c14a11d..5bb86b9 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
@@ -22,12 +22,13 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
@@ -63,9 +64,9 @@
// Declare fields.
int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
- typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
// Declare field serdes.
ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
IntegerSerializerDeserializer.INSTANCE };
@@ -73,7 +74,7 @@
// Declare keys.
int keyFieldCount = 1;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
BTree btree = BTreeUtils
.createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -140,10 +141,10 @@
// Declare fields.
int fieldCount = 3;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
- typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
- typeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[2] = IntegerPointable.TYPE_TRAITS;
// Declare field serdes.
ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -151,8 +152,8 @@
// declare keys
int keyFieldCount = 2;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+ cmps[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
BTree btree = BTreeUtils
.createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -219,9 +220,9 @@
// Declare fields.
int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
// Declare field serdes.
ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
UTF8StringSerializerDeserializer.INSTANCE };
@@ -229,7 +230,7 @@
// Declare keys.
int keyFieldCount = 1;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
BTree btree = BTreeUtils
.createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -298,9 +299,9 @@
// Declare fields.
int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
// Declare field serdes.
ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
UTF8StringSerializerDeserializer.INSTANCE };
@@ -308,7 +309,7 @@
// Declare keys.
int keyFieldCount = 1;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
BTree btree = BTreeUtils
.createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -399,9 +400,9 @@
// Declare fields.
int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
- typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+ typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
// Declare field serdes.
ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
UTF8StringSerializerDeserializer.INSTANCE };
@@ -409,7 +410,7 @@
// Declare keys.
int keyFieldCount = 1;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
BTree btree = BTreeUtils
.createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -483,10 +484,10 @@
}
// Declare fields.
int fieldCount = 3;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
- typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
- typeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[2] = IntegerPointable.TYPE_TRAITS;
// Declare field serdes.
ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -494,8 +495,8 @@
// declare keys
int keyFieldCount = 2;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+ cmps[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
BTree btree = BTreeUtils
.createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index ac85afd..e8aa00a 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -27,16 +27,16 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
@@ -96,17 +96,17 @@
// declare fields
int fieldCount = 3;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
- typeTraits[2] = new TypeTrait(4);
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[2] = IntegerPointable.TYPE_TRAITS;
// declare keys
int keyFieldCount = 3;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- cmps[2] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+ cmps[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+ cmps[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
MultiComparator cmp = new MultiComparator(cmps);
// just for printing
@@ -120,7 +120,7 @@
try {
ITreeIndexTupleWriter tupleWriter = new TypeAwareTupleWriter(typeTraits);
- BTreeFieldPrefixNSMLeafFrame frame = new BTreeFieldPrefixNSMLeafFrame(tupleWriter);
+ BTreeFieldPrefixNSMLeafFrame frame = new BTreeFieldPrefixNSMLeafFrame(tupleWriter);
frame.setPage(page);
frame.initBuffer((byte) 0);
frame.setMultiComparator(cmp);
@@ -148,7 +148,7 @@
int a = rnd.nextInt() % smallMax;
int b = rnd.nextInt() % smallMax;
int c = i;
-
+
ITupleReference tuple = createTuple(ctx, a, b, c, false);
try {
int targetTupleIndex = frame.findInsertTupleIndex(tuple);
@@ -213,19 +213,19 @@
bufferCache.unpin(page);
}
}
-
+
public int getPageSize() {
return PAGE_SIZE;
}
-
+
public int getNumPages() {
return NUM_PAGES;
}
-
+
public int getHyracksFrameSize() {
return HYRACKS_FRAME_SIZE;
}
-
+
public int getMaxOpenFiles() {
return MAX_OPEN_FILES;
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 922c715..4031e7f 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -29,13 +29,13 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
@@ -60,434 +60,390 @@
import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
public class RangeSearchCursorTest extends AbstractBTreeTest {
- // Declare fields
- int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ // Declare fields
+ int fieldCount = 2;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
- typeTraits);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- Random rnd = new Random(50);
+ Random rnd = new Random(50);
- @Before
- public void setUp() throws HyracksDataException {
- super.setUp();
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
- }
+ @Before
+ public void setUp() throws HyracksDataException {
+ super.setUp();
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+ }
- @Test
- public void uniqueIndexTest() throws Exception {
+ @Test
+ public void uniqueIndexTest() throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
}
- // declare keys
- int keyFieldCount = 1;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+ // declare keys
+ int keyFieldCount = 1;
+ IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+ cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
- MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
- tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
- tupleWriterFactory);
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(btreeFileId);
- btree.open(btreeFileId);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
- ArrayTupleReference tuple = new ArrayTupleReference();
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- // generate keys
- int numKeys = 50;
- int maxKey = 1000;
- TreeSet<Integer> uniqueKeys = new TreeSet<Integer>();
- ArrayList<Integer> keys = new ArrayList<Integer>();
- while (uniqueKeys.size() < numKeys) {
- int key = rnd.nextInt() % maxKey;
- uniqueKeys.add(key);
- }
- for (Integer i : uniqueKeys) {
- keys.add(i);
- }
+ BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
- // insert keys into btree
- for (int i = 0; i < keys.size(); i++) {
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
- TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
- tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
- try {
- indexAccessor.insert(tuple);
- } catch (BTreeException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ // generate keys
+ int numKeys = 50;
+ int maxKey = 1000;
+ TreeSet<Integer> uniqueKeys = new TreeSet<Integer>();
+ ArrayList<Integer> keys = new ArrayList<Integer>();
+ while (uniqueKeys.size() < numKeys) {
+ int key = rnd.nextInt() % maxKey;
+ uniqueKeys.add(key);
+ }
+ for (Integer i : uniqueKeys) {
+ keys.add(i);
+ }
- // btree.printTree(leafFrame, interiorFrame, recDescSers);
+ // insert keys into btree
+ for (int i = 0; i < keys.size(); i++) {
- int minSearchKey = -100;
- int maxSearchKey = 100;
+ TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- // forward searches
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, false, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, false, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, true, false);
+ try {
+ indexAccessor.insert(tuple);
+ } catch (BTreeException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- // backward searches
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, false, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, false, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, true, false);
+ // btree.printTree(leafFrame, interiorFrame, recDescSers);
- btree.close();
- }
+ int minSearchKey = -100;
+ int maxSearchKey = 100;
- @Test
- public void nonUniqueIndexTest() throws Exception {
+ // forward searches
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+
+ // backward searches
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, false, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+
+ btree.close();
+ }
+
+ @Test
+ public void nonUniqueIndexTest() throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
}
- // declare keys
- int keyFieldCount = 2;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
+ // declare keys
+ int keyFieldCount = 2;
+ IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+ cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+ cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
- MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
- tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
- tupleWriterFactory);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(btreeFileId);
- btree.open(btreeFileId);
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
+
+ BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
ITreeIndexAccessor indexAccessor = btree.createAccessor();
- // generate keys
- int numKeys = 50;
- int maxKey = 10;
- ArrayList<Integer> keys = new ArrayList<Integer>();
- for (int i = 0; i < numKeys; i++) {
- int k = rnd.nextInt() % maxKey;
- keys.add(k);
- }
- Collections.sort(keys);
+ // generate keys
+ int numKeys = 50;
+ int maxKey = 10;
+ ArrayList<Integer> keys = new ArrayList<Integer>();
+ for (int i = 0; i < numKeys; i++) {
+ int k = rnd.nextInt() % maxKey;
+ keys.add(k);
+ }
+ Collections.sort(keys);
- // insert keys into btree
- for (int i = 0; i < keys.size(); i++) {
+ // insert keys into btree
+ for (int i = 0; i < keys.size(); i++) {
- TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
+ TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- try {
- indexAccessor.insert(tuple);
- } catch (BTreeException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (BTreeException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- // btree.printTree(leafFrame, interiorFrame, recDescSers);
+ // btree.printTree(leafFrame, interiorFrame, recDescSers);
- int minSearchKey = -100;
- int maxSearchKey = 100;
+ int minSearchKey = -100;
+ int maxSearchKey = 100;
- // forward searches
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, false, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, false, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, true, false);
+ // forward searches
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
- // backward searches
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, false, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, false, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, true, false);
+ // backward searches
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, false, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
- btree.close();
- }
+ btree.close();
+ }
- @Test
- public void nonUniqueFieldPrefixIndexTest() throws Exception {
+ @Test
+ public void nonUniqueFieldPrefixIndexTest() throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
}
- // declare keys
- int keyFieldCount = 2;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE ;
+ // declare keys
+ int keyFieldCount = 2;
+ IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+ cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+ cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
- MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
- tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
- tupleWriterFactory);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(btreeFileId);
- btree.open(btreeFileId);
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+ BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
ITreeIndexAccessor indexAccessor = btree.createAccessor();
- // generate keys
- int numKeys = 50;
- int maxKey = 10;
- ArrayList<Integer> keys = new ArrayList<Integer>();
- for (int i = 0; i < numKeys; i++) {
- int k = rnd.nextInt() % maxKey;
- keys.add(k);
- }
- Collections.sort(keys);
+ // generate keys
+ int numKeys = 50;
+ int maxKey = 10;
+ ArrayList<Integer> keys = new ArrayList<Integer>();
+ for (int i = 0; i < numKeys; i++) {
+ int k = rnd.nextInt() % maxKey;
+ keys.add(k);
+ }
+ Collections.sort(keys);
- // insert keys into btree
- for (int i = 0; i < keys.size(); i++) {
+ // insert keys into btree
+ for (int i = 0; i < keys.size(); i++) {
- TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
+ TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- try {
- indexAccessor.insert(tuple);
- } catch (BTreeException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (BTreeException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- // btree.printTree(leafFrame, interiorFrame, recDescSers);
+ // btree.printTree(leafFrame, interiorFrame, recDescSers);
- int minSearchKey = -100;
- int maxSearchKey = 100;
+ int minSearchKey = -100;
+ int maxSearchKey = 100;
- // forward searches
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, false, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, false, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, true, true, true, false);
+ // forward searches
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
- // backward searches
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, false, true, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, false, false);
- performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
- maxSearchKey, false, true, true, false);
+ // backward searches
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, false, true, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false, false);
+ performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
- btree.close();
- }
+ btree.close();
+ }
- public RangePredicate createRangePredicate(int lk, int hk,
- boolean isForward, boolean lowKeyInclusive,
- boolean highKeyInclusive, MultiComparator cmp) throws HyracksDataException {
+ public RangePredicate createRangePredicate(int lk, int hk, boolean isForward, boolean lowKeyInclusive,
+ boolean highKeyInclusive, MultiComparator cmp) throws HyracksDataException {
- // create tuplereferences for search keys
- ITupleReference lowKey = TupleUtils.createIntegerTuple(lk);
- ITupleReference highKey = TupleUtils.createIntegerTuple(hk);
+ // create tuplereferences for search keys
+ ITupleReference lowKey = TupleUtils.createIntegerTuple(lk);
+ ITupleReference highKey = TupleUtils.createIntegerTuple(hk);
- IBinaryComparator[] searchCmps = new IBinaryComparator[1];
- searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
- MultiComparator searchCmp = new MultiComparator(searchCmps);
+ IBinaryComparator[] searchCmps = new IBinaryComparator[1];
+ searchCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+ MultiComparator searchCmp = new MultiComparator(searchCmps);
- RangePredicate rangePred = new RangePredicate(isForward, lowKey,
- highKey, lowKeyInclusive, highKeyInclusive, searchCmp,
- searchCmp);
- return rangePred;
- }
+ RangePredicate rangePred = new RangePredicate(isForward, lowKey, highKey, lowKeyInclusive, highKeyInclusive,
+ searchCmp, searchCmp);
+ return rangePred;
+ }
- public void getExpectedResults(ArrayList<Integer> expectedResults,
- ArrayList<Integer> keys, int lk, int hk, boolean isForward,
- boolean lowKeyInclusive, boolean highKeyInclusive) {
+ public void getExpectedResults(ArrayList<Integer> expectedResults, ArrayList<Integer> keys, int lk, int hk,
+ boolean isForward, boolean lowKeyInclusive, boolean highKeyInclusive) {
- // special cases
- if (lk == hk && (!lowKeyInclusive || !highKeyInclusive))
- return;
- if (lk > hk)
- return;
+ // special cases
+ if (lk == hk && (!lowKeyInclusive || !highKeyInclusive))
+ return;
+ if (lk > hk)
+ return;
- if (isForward) {
- for (int i = 0; i < keys.size(); i++) {
- if ((lk == keys.get(i) && lowKeyInclusive)
- || (hk == keys.get(i) && highKeyInclusive)) {
- expectedResults.add(keys.get(i));
- continue;
- }
+ if (isForward) {
+ for (int i = 0; i < keys.size(); i++) {
+ if ((lk == keys.get(i) && lowKeyInclusive) || (hk == keys.get(i) && highKeyInclusive)) {
+ expectedResults.add(keys.get(i));
+ continue;
+ }
- if (lk < keys.get(i) && hk > keys.get(i)) {
- expectedResults.add(keys.get(i));
- continue;
- }
- }
- } else {
- for (int i = keys.size() - 1; i >= 0; i--) {
- if ((lk == keys.get(i) && lowKeyInclusive)
- || (hk == keys.get(i) && highKeyInclusive)) {
- expectedResults.add(keys.get(i));
- continue;
- }
+ if (lk < keys.get(i) && hk > keys.get(i)) {
+ expectedResults.add(keys.get(i));
+ continue;
+ }
+ }
+ } else {
+ for (int i = keys.size() - 1; i >= 0; i--) {
+ if ((lk == keys.get(i) && lowKeyInclusive) || (hk == keys.get(i) && highKeyInclusive)) {
+ expectedResults.add(keys.get(i));
+ continue;
+ }
- if (lk < keys.get(i) && hk > keys.get(i)) {
- expectedResults.add(keys.get(i));
- continue;
- }
- }
- }
- }
+ if (lk < keys.get(i) && hk > keys.get(i)) {
+ expectedResults.add(keys.get(i));
+ continue;
+ }
+ }
+ }
+ }
- public boolean performSearches(ArrayList<Integer> keys, BTree btree,
- IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
- int minKey, int maxKey, boolean isForward, boolean lowKeyInclusive,
- boolean highKeyInclusive, boolean printExpectedResults)
- throws Exception {
+ public boolean performSearches(ArrayList<Integer> keys, BTree btree, IBTreeLeafFrame leafFrame,
+ IBTreeInteriorFrame interiorFrame, int minKey, int maxKey, boolean isForward, boolean lowKeyInclusive,
+ boolean highKeyInclusive, boolean printExpectedResults) throws Exception {
- ArrayList<Integer> results = new ArrayList<Integer>();
- ArrayList<Integer> expectedResults = new ArrayList<Integer>();
+ ArrayList<Integer> results = new ArrayList<Integer>();
+ ArrayList<Integer> expectedResults = new ArrayList<Integer>();
- for (int i = minKey; i < maxKey; i++) {
- for (int j = minKey; j < maxKey; j++) {
+ for (int i = minKey; i < maxKey; i++) {
+ for (int j = minKey; j < maxKey; j++) {
- results.clear();
- expectedResults.clear();
+ results.clear();
+ expectedResults.clear();
- int lowKey = i;
- int highKey = j;
+ int lowKey = i;
+ int highKey = j;
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
- RangePredicate rangePred = createRangePredicate(lowKey,
- highKey, isForward, lowKeyInclusive, highKeyInclusive,
- btree.getMultiComparator());
- ITreeIndexAccessor indexAccessor = btree.createAccessor();
- indexAccessor.search(rangeCursor, rangePred);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
+ RangePredicate rangePred = createRangePredicate(lowKey, highKey, isForward, lowKeyInclusive,
+ highKeyInclusive, btree.getMultiComparator());
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference frameTuple = rangeCursor.getTuple();
- ByteArrayInputStream inStream = new ByteArrayInputStream(
- frameTuple.getFieldData(0), frameTuple
- .getFieldStart(0), frameTuple
- .getFieldLength(0));
- DataInput dataIn = new DataInputStream(inStream);
- Integer res = IntegerSerializerDeserializer.INSTANCE
- .deserialize(dataIn);
- results.add(res);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- rangeCursor.close();
- }
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ITupleReference frameTuple = rangeCursor.getTuple();
+ ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
+ frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
+ DataInput dataIn = new DataInputStream(inStream);
+ Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
+ results.add(res);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ rangeCursor.close();
+ }
- getExpectedResults(expectedResults, keys, lowKey, highKey,
- isForward, lowKeyInclusive, highKeyInclusive);
+ getExpectedResults(expectedResults, keys, lowKey, highKey, isForward, lowKeyInclusive, highKeyInclusive);
- if (printExpectedResults) {
- if (expectedResults.size() > 0) {
- char l, u;
+ if (printExpectedResults) {
+ if (expectedResults.size() > 0) {
+ char l, u;
- if (lowKeyInclusive)
- l = '[';
- else
- l = '(';
+ if (lowKeyInclusive)
+ l = '[';
+ else
+ l = '(';
- if (highKeyInclusive)
- u = ']';
- else
- u = ')';
+ if (highKeyInclusive)
+ u = ']';
+ else
+ u = ')';
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("RANGE: " + l + " " + lowKey + " , " + highKey + " " + u);
}
- StringBuilder strBuilder = new StringBuilder();
- for (Integer r : expectedResults) {
- strBuilder.append(r + " ");
- }
+ StringBuilder strBuilder = new StringBuilder();
+ for (Integer r : expectedResults) {
+ strBuilder.append(r + " ");
+ }
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(strBuilder.toString());
}
- }
- }
+ }
+ }
- if (results.size() == expectedResults.size()) {
- for (int k = 0; k < results.size(); k++) {
- if (!results.get(k).equals(expectedResults.get(k))) {
+ if (results.size() == expectedResults.size()) {
+ for (int k = 0; k < results.size(); k++) {
+ if (!results.get(k).equals(expectedResults.get(k))) {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("DIFFERENT RESULTS AT: i=" + i + " j=" + j + " k=" + k);
LOGGER.info(results.get(k) + " " + expectedResults.get(k));
}
- return false;
- }
- }
- } else {
+ return false;
+ }
+ }
+ } else {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i + " j=" + j);
LOGGER.info("RESULTS: " + results.size());
LOGGER.info("EXPECTED RESULTS: " + expectedResults.size());
}
- return false;
- }
- }
- }
+ return false;
+ }
+ }
+ }
- return true;
- }
+ return true;
+ }
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
index dc524d7..1ae80d6 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
@@ -12,15 +12,15 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -71,14 +71,14 @@
// declare fields
int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
// declare keys
int keyFieldCount = 1;
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+ cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
index 7a61adc..5d43ae1 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
@@ -7,12 +7,12 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -35,28 +35,28 @@
import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
public class UpdateSearchTest extends AbstractBTreeTest {
-
+
// Update scan test on fixed-length tuples.
@Test
public void test01() throws Exception {
// declare fields
int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
// declare keys
int keyFieldCount = 1;
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-
+ cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
IntegerSerializerDeserializer.INSTANCE };
-
+
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
@@ -79,7 +79,7 @@
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference insertTuple = new ArrayTupleReference();
ITreeIndexAccessor indexAccessor = btree.createAccessor();
-
+
int numInserts = 10000;
for (int i = 0; i < 10000; i++) {
int f0 = rnd.nextInt() % 10000;
@@ -104,7 +104,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("DURATION: " + duration);
}
-
+
// Update scan.
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("UPDATE SCAN:");
@@ -125,7 +125,7 @@
} finally {
updateScanCursor.close();
}
-
+
// Ordered scan to verify the values.
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("ORDERED SCAN:");
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
index ca543a2..b5186b0 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
@@ -15,7 +15,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
@@ -46,7 +46,7 @@
private static final Logger LOGGER = Logger.getLogger(BTreeTestUtils.class.getName());
public static BTreeTestContext createBTreeTestContext(IBufferCache bufferCache, int btreeFileId, ISerializerDeserializer[] fieldSerdes, int numKeyFields, BTreeLeafFrameType leafType) throws Exception {
- ITypeTrait[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, fieldSerdes.length);
+ ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, fieldSerdes.length);
IBinaryComparator[] cmps = SerdeUtils.serdesToComparators(fieldSerdes, numKeyFields);
BTree btree = BTreeUtils.createBTree(bufferCache, btreeFileId, typeTraits, cmps, leafType);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index 1f1cc25..7b6bd9f 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -28,17 +28,17 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
@@ -81,8 +81,8 @@
protected int btreeFileId;
// declare token type traits
- protected ITypeTrait[] tokenTypeTraits = new ITypeTrait[] { ITypeTrait.VARLEN_TYPE_TRAIT };
- protected ITypeTrait[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
+ protected ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+ protected ITypeTraits[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
// declare btree keys
protected int btreeKeyFieldCount = 1;
@@ -109,7 +109,7 @@
protected int invListsFileId;
protected int invListFields = 1;
- protected ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
+ protected ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFields];
protected int invListKeys = 1;
protected IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
@@ -169,11 +169,12 @@
btreeFileId = fmp.lookupFileId(btreeFile);
bufferCache.openFile(btreeFileId);
- btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ btreeBinCmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory,
+ leafFrameFactory);
btree.create(btreeFileId);
btree.open(btreeFileId);
@@ -183,8 +184,8 @@
invListsFileId = fmp.lookupFileId(invListsFile);
bufferCache.openFile(invListsFileId);
- invListTypeTraits[0] = new TypeTrait(4);
- invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ invListTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ invListBinCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmp);
invIndex.open(invListsFileId);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index bbbb743..93934ed 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -31,17 +31,17 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -104,13 +104,13 @@
bufferCache.openFile(invListsFileId);
// Declare token type traits, and compute BTree type traits.
- ITypeTrait[] tokenTypeTraits = new ITypeTrait[] { ITypeTrait.VARLEN_TYPE_TRAIT };
- ITypeTrait[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
+ ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+ ITypeTraits[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
// declare btree keys
int keyFieldCount = 1;
IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
MultiComparator btreeCmp = new MultiComparator(cmps);
@@ -123,17 +123,18 @@
IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- BTree btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ BTree btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory,
+ leafFrameFactory);
btree.create(btreeFileId);
btree.open(btreeFileId);
int invListFields = 1;
- ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
- invListTypeTraits[0] = new TypeTrait(4);
+ ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFields];
+ invListTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
int invListKeys = 1;
IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
- invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ invListBinCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
MultiComparator invListCmp = new MultiComparator(invListBinCmps);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
index 0b49ec1..9c7ec09 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
@@ -23,56 +23,53 @@
import org.junit.Test;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeFrameTupleAccessor;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeFrameTupleAppender;
public class FixedSizeFrameTupleTest {
- private static int FRAME_SIZE = 4096;
+ private static int FRAME_SIZE = 4096;
- private Random rnd = new Random(50);
+ private Random rnd = new Random(50);
- /**
- * This test verifies the correct behavior of the FixedSizeFrameTuple class.
- * Frames containing FixedSizeFrameTuple's require neither tuple slots nor
- * field slots. The tests inserts generated data into a frame until the
- * frame is full, and then verifies the frame's contents.
- *
- */
- @Test
- public void singleFieldTest() throws Exception {
- ByteBuffer buffer = ByteBuffer.allocate(FRAME_SIZE);
+ /**
+ * This test verifies the correct behavior of the FixedSizeFrameTuple class.
+ * Frames containing FixedSizeFrameTuple's require neither tuple slots nor
+ * field slots. The tests inserts generated data into a frame until the
+ * frame is full, and then verifies the frame's contents.
+ *
+ */
+ @Test
+ public void singleFieldTest() throws Exception {
+ ByteBuffer buffer = ByteBuffer.allocate(FRAME_SIZE);
- ITypeTrait[] fields = new TypeTrait[1];
- fields[0] = new TypeTrait(4);
+ ITypeTraits[] fields = new ITypeTraits[1];
+ fields[0] = IntegerPointable.TYPE_TRAITS;
- FixedSizeFrameTupleAppender ftapp = new FixedSizeFrameTupleAppender(
- FRAME_SIZE, fields);
- FixedSizeFrameTupleAccessor ftacc = new FixedSizeFrameTupleAccessor(
- FRAME_SIZE, fields);
+ FixedSizeFrameTupleAppender ftapp = new FixedSizeFrameTupleAppender(FRAME_SIZE, fields);
+ FixedSizeFrameTupleAccessor ftacc = new FixedSizeFrameTupleAccessor(FRAME_SIZE, fields);
- boolean frameHasSpace = true;
+ boolean frameHasSpace = true;
- ArrayList<Integer> check = new ArrayList<Integer>();
+ ArrayList<Integer> check = new ArrayList<Integer>();
- ftapp.reset(buffer, true);
- while (frameHasSpace) {
- int val = rnd.nextInt();
- frameHasSpace = ftapp.append(val);
- if (frameHasSpace) {
- check.add(val);
- ftapp.incrementTupleCount(1);
- }
- }
+ ftapp.reset(buffer, true);
+ while (frameHasSpace) {
+ int val = rnd.nextInt();
+ frameHasSpace = ftapp.append(val);
+ if (frameHasSpace) {
+ check.add(val);
+ ftapp.incrementTupleCount(1);
+ }
+ }
- ftacc.reset(buffer);
- for (int i = 0; i < ftacc.getTupleCount(); i++) {
- int val = IntegerSerializerDeserializer.getInt(ftacc.getBuffer()
- .array(), ftacc.getTupleStartOffset(i));
- Assert.assertEquals(check.get(i).intValue(), val);
- }
- }
+ ftacc.reset(buffer);
+ for (int i = 0; i < ftacc.getTupleCount(); i++) {
+ int val = IntegerSerializerDeserializer.getInt(ftacc.getBuffer().array(), ftacc.getTupleStartOffset(i));
+ Assert.assertEquals(check.get(i).intValue(), val);
+ }
+ }
}
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index faf1e88..9fe2c94 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -27,17 +27,17 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
@@ -66,792 +66,704 @@
import edu.uci.ics.hyracks.test.support.TestUtils;
public class RTreeTest extends AbstractRTreeTest {
- private static final int PAGE_SIZE = 256;
- private static final int NUM_PAGES = 10;
- private static final int MAX_OPEN_FILES = 10;
- private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private static final int PAGE_SIZE = 256;
+ private static final int NUM_PAGES = 10;
+ private static final int MAX_OPEN_FILES = 10;
+ private static final int HYRACKS_FRAME_SIZE = 128;
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
- // create an R-tree of two dimensions
- // fill the R-tree with random values using insertions
- // perform ordered scan
- @Test
- public void test01() throws Exception {
+ // create an R-tree of two dimensions
+ // fill the R-tree with random values using insertions
+ // perform ordered scan
+ @Test
+ public void test01() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- // declare keys
- int keyFieldCount = 4;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
- cmps[1] = cmps[0];
- cmps[2] = cmps[0];
- cmps[3] = cmps[0];
+ // declare keys
+ int keyFieldCount = 4;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+ cmps[1] = cmps[0];
+ cmps[2] = cmps[0];
+ cmps[3] = cmps[0];
- // declare tuple fields
- int fieldCount = 7;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(8);
- typeTraits[1] = new TypeTrait(8);
- typeTraits[2] = new TypeTrait(8);
- typeTraits[3] = new TypeTrait(8);
- typeTraits[4] = new TypeTrait(8);
- typeTraits[5] = new TypeTrait(4);
- typeTraits[6] = new TypeTrait(8);
+ // declare tuple fields
+ int fieldCount = 7;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = DoublePointable.TYPE_TRAITS;
+ typeTraits[1] = DoublePointable.TYPE_TRAITS;
+ typeTraits[2] = DoublePointable.TYPE_TRAITS;
+ typeTraits[3] = DoublePointable.TYPE_TRAITS;
+ typeTraits[4] = DoublePointable.TYPE_TRAITS;
+ typeTraits[5] = DoublePointable.TYPE_TRAITS;
+ typeTraits[6] = DoublePointable.TYPE_TRAITS;
- // create value providers
- IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps);
+ // create value providers
+ IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+ cmps.length, DoublePointable.FACTORY);
- MultiComparator cmp = new MultiComparator(cmps);
+ MultiComparator cmp = new MultiComparator(cmps);
- RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- typeTraits);
+ RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
- .createFrame();
- IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, fileId, 0, metaFrameFactory);
+ IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+ IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
- RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
- interiorFrameFactory, leafFrameFactory);
- rtree.create(fileId);
- rtree.open(fileId);
+ RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ rtree.create(fileId);
+ rtree.open(fileId);
- ByteBuffer hyracksFrame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
+ ByteBuffer hyracksFrame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] recDescSers = {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(
- ctx.getFrameSize(), recDesc);
- accessor.reset(hyracksFrame);
- FrameTupleReference tuple = new FrameTupleReference();
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(hyracksFrame);
+ FrameTupleReference tuple = new FrameTupleReference();
- ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
- Random rnd = new Random();
- rnd.setSeed(50);
+ Random rnd = new Random();
+ rnd.setSeed(50);
- Random rnd2 = new Random();
- rnd2.setSeed(50);
- for (int i = 0; i < 5000; i++) {
+ Random rnd2 = new Random();
+ rnd2.setSeed(50);
+ for (int i = 0; i < 5000; i++) {
- double p1x = rnd.nextDouble();
- double p1y = rnd.nextDouble();
- double p2x = rnd.nextDouble();
- double p2y = rnd.nextDouble();
+ double p1x = rnd.nextDouble();
+ double p1y = rnd.nextDouble();
+ double p2x = rnd.nextDouble();
+ double p2y = rnd.nextDouble();
- double pk1 = rnd2.nextDouble();
- int pk2 = rnd2.nextInt();
- double pk3 = rnd2.nextDouble();
+ double pk1 = rnd2.nextDouble();
+ int pk2 = rnd2.nextInt();
+ double pk3 = rnd2.nextDouble();
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x)
- + " " + Math.min(p1y, p2y) + " "
- + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
- }
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
+ }
- try {
- indexAccessor.insert(tuple);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- String rtreeStats = rtree.printStats();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rtreeStats);
- }
+ String rtreeStats = rtree.printStats();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
- // disk-order scan
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("DISK-ORDER SCAN:");
- }
- TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
- leafFrame);
- indexAccessor.diskOrderScan(diskOrderCursor);
- try {
- while (diskOrderCursor.hasNext()) {
- diskOrderCursor.next();
- ITupleReference frameTuple = diskOrderCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rec);
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- diskOrderCursor.close();
- }
+ // disk-order scan
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
+ TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+ indexAccessor.diskOrderScan(diskOrderCursor);
+ try {
+ while (diskOrderCursor.hasNext()) {
+ diskOrderCursor.next();
+ ITupleReference frameTuple = diskOrderCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ diskOrderCursor.close();
+ }
- TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
- bufferCache, freePageManager, fileId, rtree.getRootPageId());
- TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
- interiorFrame, metaFrame);
- String string = stats.toString();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(string);
- }
+ TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+ rtree.getRootPageId());
+ TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+ String string = stats.toString();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
- rtree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
+ rtree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
- }
+ }
- // create an R-tree of two dimensions
- // fill the R-tree with random values using insertions
- // and then delete all the tuples which result of an empty R-tree
- @Test
- public void test02() throws Exception {
+ // create an R-tree of two dimensions
+ // fill the R-tree with random values using insertions
+ // and then delete all the tuples which result of an empty R-tree
+ @Test
+ public void test02() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- // declare keys
- int keyFieldCount = 4;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
- cmps[1] = cmps[0];
- cmps[2] = cmps[0];
- cmps[3] = cmps[0];
+ // declare keys
+ int keyFieldCount = 4;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+ cmps[1] = cmps[0];
+ cmps[2] = cmps[0];
+ cmps[3] = cmps[0];
- // declare tuple fields
- int fieldCount = 7;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(8);
- typeTraits[1] = new TypeTrait(8);
- typeTraits[2] = new TypeTrait(8);
- typeTraits[3] = new TypeTrait(8);
- typeTraits[4] = new TypeTrait(8);
- typeTraits[5] = new TypeTrait(4);
- typeTraits[6] = new TypeTrait(8);
+ // declare tuple fields
+ int fieldCount = 7;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = DoublePointable.TYPE_TRAITS;
+ typeTraits[1] = DoublePointable.TYPE_TRAITS;
+ typeTraits[2] = DoublePointable.TYPE_TRAITS;
+ typeTraits[3] = DoublePointable.TYPE_TRAITS;
+ typeTraits[4] = DoublePointable.TYPE_TRAITS;
+ typeTraits[5] = DoublePointable.TYPE_TRAITS;
+ typeTraits[6] = DoublePointable.TYPE_TRAITS;
- // create value providers
- IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps);
+ // create value providers
+ IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+ cmps.length, DoublePointable.FACTORY);
- MultiComparator cmp = new MultiComparator(cmps);
+ MultiComparator cmp = new MultiComparator(cmps);
- RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- typeTraits);
+ RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
- .createFrame();
- IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, fileId, 0, metaFrameFactory);
+ IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+ IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
- RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
- interiorFrameFactory, leafFrameFactory);
- rtree.create(fileId);
- rtree.open(fileId);
+ RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ rtree.create(fileId);
+ rtree.open(fileId);
- ByteBuffer hyracksFrame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
+ ByteBuffer hyracksFrame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] recDescSers = {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(
- ctx.getFrameSize(), recDesc);
- accessor.reset(hyracksFrame);
- FrameTupleReference tuple = new FrameTupleReference();
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(hyracksFrame);
+ FrameTupleReference tuple = new FrameTupleReference();
- ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
- Random rnd = new Random();
- rnd.setSeed(50);
+ Random rnd = new Random();
+ rnd.setSeed(50);
- for (int i = 0; i < 5000; i++) {
+ for (int i = 0; i < 5000; i++) {
- double p1x = rnd.nextDouble();
- double p1y = rnd.nextDouble();
- double p2x = rnd.nextDouble();
- double p2y = rnd.nextDouble();
+ double p1x = rnd.nextDouble();
+ double p1y = rnd.nextDouble();
+ double p2x = rnd.nextDouble();
+ double p2y = rnd.nextDouble();
- double pk1 = rnd.nextDouble();
- int pk2 = rnd.nextInt();
- double pk3 = rnd.nextDouble();
+ double pk1 = rnd.nextDouble();
+ int pk2 = rnd.nextInt();
+ double pk3 = rnd.nextDouble();
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
- + " " + Math.max(p1y, p2y));
- }
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
+ }
- try {
- indexAccessor.insert(tuple);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- String rtreeStats = rtree.printStats();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rtreeStats);
- }
+ String rtreeStats = rtree.printStats();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
- rnd.setSeed(50);
- for (int i = 0; i < 5000; i++) {
+ rnd.setSeed(50);
+ for (int i = 0; i < 5000; i++) {
- double p1x = rnd.nextDouble();
- double p1y = rnd.nextDouble();
- double p2x = rnd.nextDouble();
- double p2y = rnd.nextDouble();
+ double p1x = rnd.nextDouble();
+ double p1y = rnd.nextDouble();
+ double p2x = rnd.nextDouble();
+ double p2y = rnd.nextDouble();
- double pk1 = rnd.nextDouble();
- int pk2 = rnd.nextInt();
- double pk3 = rnd.nextDouble();
+ double pk1 = rnd.nextDouble();
+ int pk2 = rnd.nextInt();
+ double pk3 = rnd.nextDouble();
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("DELETING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
- + " " + Math.max(p1y, p2y));
- }
- }
+ tuple.reset(accessor, 0);
- try {
- indexAccessor.delete(tuple);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("DELETING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
+ }
- TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
- bufferCache, freePageManager, fileId, rtree.getRootPageId());
- TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
- interiorFrame, metaFrame);
- String string = stats.toString();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(string);
- }
+ try {
+ indexAccessor.delete(tuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+ rtree.getRootPageId());
+ TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+ String string = stats.toString();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
- rtree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
+ rtree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
- }
+ }
- // create an R-tree of three dimensions
- // fill the R-tree with random values using insertions
- // perform ordered scan
- @Test
- public void test03() throws Exception {
+ // create an R-tree of three dimensions
+ // fill the R-tree with random values using insertions
+ // perform ordered scan
+ @Test
+ public void test03() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- // declare keys
- int keyFieldCount = 6;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
- cmps[1] = cmps[0];
- cmps[2] = cmps[0];
- cmps[3] = cmps[0];
- cmps[4] = cmps[0];
- cmps[5] = cmps[0];
+ // declare keys
+ int keyFieldCount = 6;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+ cmps[1] = cmps[0];
+ cmps[2] = cmps[0];
+ cmps[3] = cmps[0];
+ cmps[4] = cmps[0];
+ cmps[5] = cmps[0];
- // declare tuple fields
- int fieldCount = 9;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(8);
- typeTraits[1] = new TypeTrait(8);
- typeTraits[2] = new TypeTrait(8);
- typeTraits[3] = new TypeTrait(8);
- typeTraits[4] = new TypeTrait(8);
- typeTraits[5] = new TypeTrait(8);
- typeTraits[6] = new TypeTrait(8);
- typeTraits[7] = new TypeTrait(4);
- typeTraits[8] = new TypeTrait(8);
+ // declare tuple fields
+ int fieldCount = 9;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = DoublePointable.TYPE_TRAITS;
+ typeTraits[1] = DoublePointable.TYPE_TRAITS;
+ typeTraits[2] = DoublePointable.TYPE_TRAITS;
+ typeTraits[3] = DoublePointable.TYPE_TRAITS;
+ typeTraits[4] = DoublePointable.TYPE_TRAITS;
+ typeTraits[5] = DoublePointable.TYPE_TRAITS;
+ typeTraits[6] = DoublePointable.TYPE_TRAITS;
+ typeTraits[7] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[8] = DoublePointable.TYPE_TRAITS;
- // create value providers
- IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps);
+ // create value providers
+ IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+ cmps.length, DoublePointable.FACTORY);
- MultiComparator cmp = new MultiComparator(cmps);
+ MultiComparator cmp = new MultiComparator(cmps);
- RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- typeTraits);
+ RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
- .createFrame();
- IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, fileId, 0, metaFrameFactory);
+ IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+ IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
- RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
- interiorFrameFactory, leafFrameFactory);
- rtree.create(fileId);
- rtree.open(fileId);
+ RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ rtree.create(fileId);
+ rtree.open(fileId);
- ByteBuffer hyracksFrame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
+ ByteBuffer hyracksFrame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] recDescSers = {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(
- ctx.getFrameSize(), recDesc);
- accessor.reset(hyracksFrame);
- FrameTupleReference tuple = new FrameTupleReference();
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(hyracksFrame);
+ FrameTupleReference tuple = new FrameTupleReference();
- ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
- Random rnd = new Random();
- rnd.setSeed(50);
+ Random rnd = new Random();
+ rnd.setSeed(50);
- for (int i = 0; i < 5000; i++) {
+ for (int i = 0; i < 5000; i++) {
- double p1x = rnd.nextDouble();
- double p1y = rnd.nextDouble();
- double p1z = rnd.nextDouble();
- double p2x = rnd.nextDouble();
- double p2y = rnd.nextDouble();
- double p2z = rnd.nextDouble();
+ double p1x = rnd.nextDouble();
+ double p1y = rnd.nextDouble();
+ double p1z = rnd.nextDouble();
+ double p2x = rnd.nextDouble();
+ double p2y = rnd.nextDouble();
+ double p2z = rnd.nextDouble();
- double pk1 = rnd.nextDouble();
- int pk2 = rnd.nextInt();
- double pk3 = rnd.nextDouble();
+ double pk1 = rnd.nextDouble();
+ int pk2 = rnd.nextInt();
+ double pk3 = rnd.nextDouble();
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1z, p2z),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1z, p2z),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1z, p2z), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1z, p2z), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.min(p1z, p2z)
- + " " + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + " " + Math.max(p1z, p2z));
- }
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.min(p1z, p2z) + " " + " " + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y) + " "
+ + Math.max(p1z, p2z));
+ }
+ }
- try {
- indexAccessor.insert(tuple);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- String rtreeStats = rtree.printStats();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rtreeStats);
- }
+ String rtreeStats = rtree.printStats();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
- // disk-order scan
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("DISK-ORDER SCAN:");
- }
- TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
- leafFrame);
- indexAccessor.diskOrderScan(diskOrderCursor);
- try {
- while (diskOrderCursor.hasNext()) {
- diskOrderCursor.next();
- ITupleReference frameTuple = diskOrderCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rec);
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- diskOrderCursor.close();
- }
+ // disk-order scan
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
+ TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+ indexAccessor.diskOrderScan(diskOrderCursor);
+ try {
+ while (diskOrderCursor.hasNext()) {
+ diskOrderCursor.next();
+ ITupleReference frameTuple = diskOrderCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ diskOrderCursor.close();
+ }
- TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
- bufferCache, freePageManager, fileId, rtree.getRootPageId());
- TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
- interiorFrame, metaFrame);
- String string = stats.toString();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(string);
- }
+ TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+ rtree.getRootPageId());
+ TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+ String string = stats.toString();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
- rtree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
+ rtree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
- }
+ }
- // create an R-tree of two dimensions
- // fill the R-tree with random integer key values using insertions
- // perform ordered scan
- @Test
- public void test04() throws Exception {
+ // create an R-tree of two dimensions
+ // fill the R-tree with random integer key values using insertions
+ // perform ordered scan
+ @Test
+ public void test04() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- // declare keys
- int keyFieldCount = 4;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = IntegerBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
- cmps[1] = cmps[0];
- cmps[2] = cmps[0];
- cmps[3] = cmps[0];
+ // declare keys
+ int keyFieldCount = 4;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+ cmps[1] = cmps[0];
+ cmps[2] = cmps[0];
+ cmps[3] = cmps[0];
- // declare tuple fields
- int fieldCount = 7;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
- typeTraits[2] = new TypeTrait(4);
- typeTraits[3] = new TypeTrait(4);
- typeTraits[4] = new TypeTrait(8);
- typeTraits[5] = new TypeTrait(4);
- typeTraits[6] = new TypeTrait(8);
+ // declare tuple fields
+ int fieldCount = 7;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[3] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[4] = DoublePointable.TYPE_TRAITS;
+ typeTraits[5] = IntegerPointable.TYPE_TRAITS;
+ typeTraits[6] = DoublePointable.TYPE_TRAITS;
- // create value providers
- IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps);
+ // create value providers
+ IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+ cmps.length, IntegerPointable.FACTORY);
- MultiComparator cmp = new MultiComparator(cmps);
+ MultiComparator cmp = new MultiComparator(cmps);
- RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- typeTraits);
+ RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
- .createFrame();
- IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, fileId, 0, metaFrameFactory);
+ IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+ IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
- RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
- interiorFrameFactory, leafFrameFactory);
- rtree.create(fileId);
- rtree.open(fileId);
+ RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ rtree.create(fileId);
+ rtree.open(fileId);
- ByteBuffer hyracksFrame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
+ ByteBuffer hyracksFrame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] recDescSers = {
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(
- ctx.getFrameSize(), recDesc);
- accessor.reset(hyracksFrame);
- FrameTupleReference tuple = new FrameTupleReference();
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(hyracksFrame);
+ FrameTupleReference tuple = new FrameTupleReference();
- ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
- Random rnd = new Random();
- rnd.setSeed(50);
+ Random rnd = new Random();
+ rnd.setSeed(50);
- Random rnd2 = new Random();
- rnd2.setSeed(50);
- for (int i = 0; i < 5000; i++) {
+ Random rnd2 = new Random();
+ rnd2.setSeed(50);
+ for (int i = 0; i < 5000; i++) {
- int p1x = rnd.nextInt();
- int p1y = rnd.nextInt();
- int p2x = rnd.nextInt();
- int p2y = rnd.nextInt();
+ int p1x = rnd.nextInt();
+ int p1y = rnd.nextInt();
+ int p2x = rnd.nextInt();
+ int p2y = rnd.nextInt();
- double pk1 = rnd2.nextDouble();
- int pk2 = rnd2.nextInt();
- double pk3 = rnd2.nextDouble();
+ double pk1 = rnd2.nextDouble();
+ int pk2 = rnd2.nextInt();
+ double pk3 = rnd2.nextDouble();
- tb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(
- Math.min(p1x, p2x), dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(
- Math.min(p1y, p2y), dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(
- Math.max(p1x, p2x), dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(
- Math.max(p1y, p2y), dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
- + " " + Math.max(p1y, p2y));
- }
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
+ }
- try {
- indexAccessor.insert(tuple);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- String rtreeStats = rtree.printStats();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rtreeStats);
- }
+ String rtreeStats = rtree.printStats();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
- // disk-order scan
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("DISK-ORDER SCAN:");
- }
- TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
- leafFrame);
- indexAccessor.diskOrderScan(diskOrderCursor);
- try {
- while (diskOrderCursor.hasNext()) {
- diskOrderCursor.next();
- ITupleReference frameTuple = diskOrderCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(rec);
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- diskOrderCursor.close();
- }
+ // disk-order scan
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
+ TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+ indexAccessor.diskOrderScan(diskOrderCursor);
+ try {
+ while (diskOrderCursor.hasNext()) {
+ diskOrderCursor.next();
+ ITupleReference frameTuple = diskOrderCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ diskOrderCursor.close();
+ }
- TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
- bufferCache, freePageManager, fileId, rtree.getRootPageId());
- TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
- interiorFrame, metaFrame);
- String string = stats.toString();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(string);
- }
+ TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+ rtree.getRootPageId());
+ TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+ String string = stats.toString();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
- rtree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
+ rtree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
- }
+ }
}
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index 60d0578..a232e37 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -31,16 +31,17 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
@@ -69,212 +70,183 @@
import edu.uci.ics.hyracks.test.support.TestUtils;
public class SearchCursorTest extends AbstractRTreeTest {
- private static final int PAGE_SIZE = 256;
- private static final int NUM_PAGES = 10;
- private static final int MAX_OPEN_FILES = 10;
- private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private static final int PAGE_SIZE = 256;
+ private static final int NUM_PAGES = 10;
+ private static final int MAX_OPEN_FILES = 10;
+ private static final int HYRACKS_FRAME_SIZE = 128;
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
- // create an R-tree of two dimensions
- // fill the R-tree with random values using insertions
- // and then perform range search
- @Test
- public void searchCursorTest() throws Exception {
+ // create an R-tree of two dimensions
+ // fill the R-tree with random values using insertions
+ // and then perform range search
+ @Test
+ public void searchCursorTest() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- // declare keys
- int keyFieldCount = 4;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
- cmps[1] = cmps[0];
- cmps[2] = cmps[0];
- cmps[3] = cmps[0];
+ // declare keys
+ int keyFieldCount = 4;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+ cmps[1] = cmps[0];
+ cmps[2] = cmps[0];
+ cmps[3] = cmps[0];
- // declare tuple fields
- int fieldCount = 5;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(8);
- typeTraits[1] = new TypeTrait(8);
- typeTraits[2] = new TypeTrait(8);
- typeTraits[3] = new TypeTrait(8);
- typeTraits[4] = new TypeTrait(4);
+ // declare tuple fields
+ int fieldCount = 5;
+ ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+ typeTraits[0] = DoublePointable.TYPE_TRAITS;
+ typeTraits[1] = DoublePointable.TYPE_TRAITS;
+ typeTraits[2] = DoublePointable.TYPE_TRAITS;
+ typeTraits[3] = DoublePointable.TYPE_TRAITS;
+ typeTraits[4] = IntegerPointable.TYPE_TRAITS;
- // create value providers
- IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps);
+ // create value providers
+ IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+ cmps.length, DoublePointable.FACTORY);
- MultiComparator cmp = new MultiComparator(cmps);
+ MultiComparator cmp = new MultiComparator(cmps);
- RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- typeTraits);
+ RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
- tupleWriterFactory, valueProviderFactories);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+ valueProviderFactories);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) interiorFrameFactory
- .createFrame();
- IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory
- .createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, fileId, 0, metaFrameFactory);
+ IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) interiorFrameFactory.createFrame();
+ IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory.createFrame();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
- RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
- interiorFrameFactory, leafFrameFactory);
- rtree.create(fileId);
- rtree.open(fileId);
+ RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ rtree.create(fileId);
+ rtree.open(fileId);
- ByteBuffer hyracksFrame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
+ ByteBuffer hyracksFrame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] recDescSers = {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(
- ctx.getFrameSize(), recDesc);
- accessor.reset(hyracksFrame);
- FrameTupleReference tuple = new FrameTupleReference();
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(hyracksFrame);
+ FrameTupleReference tuple = new FrameTupleReference();
- ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
- Random rnd = new Random();
- rnd.setSeed(50);
- for (int i = 0; i < 5000; i++) {
+ Random rnd = new Random();
+ rnd.setSeed(50);
+ for (int i = 0; i < 5000; i++) {
- double p1x = rnd.nextDouble();
- double p1y = rnd.nextDouble();
- double p2x = rnd.nextDouble();
- double p2y = rnd.nextDouble();
+ double p1x = rnd.nextDouble();
+ double p1y = rnd.nextDouble();
+ double p2x = rnd.nextDouble();
+ double p2y = rnd.nextDouble();
- int pk = rnd.nextInt();
+ int pk = rnd.nextInt();
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- if (LOGGER.isLoggable(Level.INFO)) {
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
- + " " + Math.max(p1y, p2y));
- }
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
+ }
- try {
- indexAccessor.insert(tuple);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
- for (int i = 0; i < 50; i++) {
- double p1x = rnd.nextDouble();
- double p1y = rnd.nextDouble();
- double p2x = rnd.nextDouble();
- double p2y = rnd.nextDouble();
+ for (int i = 0; i < 50; i++) {
+ double p1x = rnd.nextDouble();
+ double p1y = rnd.nextDouble();
+ double p2x = rnd.nextDouble();
+ double p2y = rnd.nextDouble();
- int pk = rnd.nextInt();
+ int pk = rnd.nextInt();
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
- dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
- dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
+ tb.addFieldEndOffset();
- appender.reset(hyracksFrame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ appender.reset(hyracksFrame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(i + " Searching for: " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y));
- }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(i + " Searching for: " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
- ITreeIndexCursor searchCursor = new RTreeSearchCursor(
- interiorFrame, leafFrame);
- SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
+ ITreeIndexCursor searchCursor = new RTreeSearchCursor(interiorFrame, leafFrame);
+ SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
- indexAccessor.search(searchCursor, searchPredicate);
+ indexAccessor.search(searchCursor, searchPredicate);
- ArrayList<Integer> results = new ArrayList<Integer>();
- try {
- while (searchCursor.hasNext()) {
- searchCursor.next();
- ITupleReference frameTuple = searchCursor.getTuple();
- ByteArrayInputStream inStream = new ByteArrayInputStream(
- frameTuple.getFieldData(4),
- frameTuple.getFieldStart(4),
- frameTuple.getFieldLength(4));
- DataInput dataIn = new DataInputStream(inStream);
- Integer res = IntegerSerializerDeserializer.INSTANCE
- .deserialize(dataIn);
- results.add(res);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- searchCursor.close();
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("There are " + results.size()
- + " objects that satisfy the query");
- }
- }
+ ArrayList<Integer> results = new ArrayList<Integer>();
+ try {
+ while (searchCursor.hasNext()) {
+ searchCursor.next();
+ ITupleReference frameTuple = searchCursor.getTuple();
+ ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(4),
+ frameTuple.getFieldStart(4), frameTuple.getFieldLength(4));
+ DataInput dataIn = new DataInputStream(inStream);
+ Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
+ results.add(res);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ searchCursor.close();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("There are " + results.size() + " objects that satisfy the query");
+ }
+ }
- rtree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
+ rtree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
+ }
}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index d190af3..890f42d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,6 +79,7 @@
</pluginRepositories>
<modules>
+ <module>hyracks-ipc</module>
<module>hyracks-api</module>
<module>hyracks-dataflow-common</module>
<module>hyracks-dataflow-std</module>
@@ -86,6 +87,7 @@
<module>hyracks-control-common</module>
<module>hyracks-control-cc</module>
<module>hyracks-control-nc</module>
+ <module>hyracks-data</module>
<module>hyracks-cli</module>
<module>hyracks-storage-common</module>
<module>hyracks-storage-am-common</module>
@@ -98,5 +100,6 @@
<module>hyracks-examples</module>
<module>hyracks-documentation</module>
<module>hyracks-hadoop-compat</module>
+ <module>hyracks-algebricks</module>
</modules>
</project>