From e31d209a6c4111f49c7e53243106625fde62b6c9 Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Mon, 19 Dec 2016 13:33:34 +0800 Subject: [PATCH 01/15] implement distribute functions, refactor producers/consumers --- .../escience/myria/api/DatasetResource.java | 76 ++- .../encoding/AbstractConsumerEncoding.java | 5 + .../encoding/AbstractProducerEncoding.java | 3 + .../myria/api/encoding/AggregateEncoding.java | 2 + .../encoding/BroadcastConsumerEncoding.java | 18 +- .../encoding/BroadcastProducerEncoding.java | 16 +- .../api/encoding/CollectConsumerEncoding.java | 15 +- .../myria/api/encoding/DatasetEncoding.java | 9 +- .../myria/api/encoding/DatasetStatus.java | 54 +- .../myria/api/encoding/DbInsertEncoding.java | 13 +- .../GenericShuffleConsumerEncoding.java | 15 + ...va => GenericShuffleProducerEncoding.java} | 25 +- .../HyperCubeShuffleConsumerEncoding.java | 17 + .../HyperCubeShuffleProducerEncoding.java | 44 ++ .../HyperShuffleConsumerEncoding.java | 20 - .../HyperShuffleProducerEncoding.java | 65 --- .../LocalMultiwayConsumerEncoding.java | 19 +- .../LocalMultiwayProducerEncoding.java | 10 +- .../myria/api/encoding/OperatorEncoding.java | 28 +- .../api/encoding/ParallelDatasetEncoding.java | 6 +- .../myria/api/encoding/QueryConstruct.java | 3 +- .../api/encoding/ShuffleConsumerEncoding.java | 20 - .../api/encoding/TempInsertEncoding.java | 19 +- .../api/encoding/TipsyDatasetEncoding.java | 15 +- .../myria/coordinator/MasterCatalog.java | 51 +- .../escience/myria/operator/DbInsert.java | 68 +-- .../escience/myria/operator/DbInsertTemp.java | 51 +- .../operator/network/CollectConsumer.java | 52 -- .../operator/network/CollectProducer.java | 29 +- .../myria/operator/network/Consumer.java | 108 +--- .../myria/operator/network/EOSController.java | 2 +- .../network/GenericShuffleConsumer.java | 52 -- .../network/GenericShuffleProducer.java | 123 +---- .../network/LocalMultiwayConsumer.java | 36 -- .../network/LocalMultiwayProducer.java | 26 +- .../network/LocalShuffleConsumer.java | 26 - .../network/LocalShuffleProducer.java | 39 -- .../myria/operator/network/Producer.java | 237 ++------ .../operator/network/RecoverProducer.java | 18 +- .../BroadcastDistributeFunction.java | 36 ++ .../distribute/DistributeFunction.java | 82 +++ .../distribute/HashDistributeFunction.java | 40 ++ .../distribute/HashPartitionFunction.java | 79 +++ .../network/distribute/HowDistributed.java | 50 ++ .../HyperCubeDistributeFunction.java | 45 ++ .../HyperCubePartitionFunction.java | 57 ++ .../IdentityDistributeFunction.java | 31 ++ .../distribute/IdentityPartitionFunction.java | 42 ++ .../network/distribute/PartitionFunction.java | 50 ++ .../RoundRobinDistributeFunction.java | 28 + .../RoundRobinPartitionFunction.java | 32 ++ .../distribute/SinglePartitionFunction.java | 26 + .../network/distribute/package-info.java | 4 + .../partition/FixValuePartitionFunction.java | 39 -- .../network/partition/HowPartitioned.java | 58 -- .../IdentityHashPartitionFunction.java | 64 --- .../partition/MFMDHashPartitionFunction.java | 65 --- .../MultiFieldHashPartitionFunction.java | 85 --- .../network/partition/PartitionFunction.java | 80 --- .../RoundRobinPartitionFunction.java | 41 -- .../SingleFieldHashPartitionFunction.java | 82 --- .../WholeTupleHashPartitionFunction.java | 41 -- .../network/partition/package-info.java | 4 - .../myria/parallel/RelationWriteMetadata.java | 78 +-- .../escience/myria/parallel/Server.java | 504 ++++++------------ .../escience/myria/storage/TupleBatch.java | 122 ++--- .../escience/myria/util/HashUtils.java | 20 +- .../escience/myria/util/MyriaArrayUtils.java | 80 +-- .../myria/api/SerializationTests.java | 66 +-- .../MultiFieldHashPartitionFunctionTest.java | 25 +- .../escience/myria/util/TestUtils.java | 75 +-- ...a_two_col_int_unix_no_trailing_newline.txt | 2 +- 72 files changed, 1322 insertions(+), 2246 deletions(-) create mode 100644 src/edu/washington/escience/myria/api/encoding/GenericShuffleConsumerEncoding.java rename src/edu/washington/escience/myria/api/encoding/{ShuffleProducerEncoding.java => GenericShuffleProducerEncoding.java} (51%) create mode 100644 src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleConsumerEncoding.java create mode 100644 src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java delete mode 100644 src/edu/washington/escience/myria/api/encoding/HyperShuffleConsumerEncoding.java delete mode 100644 src/edu/washington/escience/myria/api/encoding/HyperShuffleProducerEncoding.java delete mode 100644 src/edu/washington/escience/myria/api/encoding/ShuffleConsumerEncoding.java delete mode 100644 src/edu/washington/escience/myria/operator/network/CollectConsumer.java delete mode 100644 src/edu/washington/escience/myria/operator/network/GenericShuffleConsumer.java delete mode 100644 src/edu/washington/escience/myria/operator/network/LocalMultiwayConsumer.java delete mode 100644 src/edu/washington/escience/myria/operator/network/LocalShuffleConsumer.java delete mode 100644 src/edu/washington/escience/myria/operator/network/LocalShuffleProducer.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/SinglePartitionFunction.java create mode 100644 src/edu/washington/escience/myria/operator/network/distribute/package-info.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/FixValuePartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/HowPartitioned.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/IdentityHashPartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/MFMDHashPartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/MultiFieldHashPartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/PartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/RoundRobinPartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/SingleFieldHashPartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/WholeTupleHashPartitionFunction.java delete mode 100644 src/edu/washington/escience/myria/operator/network/partition/package-info.java diff --git a/src/edu/washington/escience/myria/api/DatasetResource.java b/src/edu/washington/escience/myria/api/DatasetResource.java index 36266e9b1..4636c27de 100644 --- a/src/edu/washington/escience/myria/api/DatasetResource.java +++ b/src/edu/washington/escience/myria/api/DatasetResource.java @@ -4,7 +4,6 @@ import java.io.InputStream; import java.net.URI; import java.util.List; -import java.util.Set; import javax.annotation.Nonnull; import javax.ws.rs.Consumes; @@ -59,16 +58,13 @@ import edu.washington.escience.myria.operator.Operator; import edu.washington.escience.myria.operator.TipsyFileScan; import edu.washington.escience.myria.operator.TupleSource; -import edu.washington.escience.myria.operator.network.partition.HowPartitioned; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; -import edu.washington.escience.myria.operator.network.partition.RoundRobinPartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.HowDistributed; +import edu.washington.escience.myria.operator.network.distribute.RoundRobinDistributeFunction; import edu.washington.escience.myria.parallel.Server; import edu.washington.escience.myria.storage.TupleBatch; -/** - * This is the class that handles API calls to create or fetch datasets. - * - */ +/** This is the class that handles API calls to create or fetch datasets. */ @Consumes(MediaType.APPLICATION_JSON) @Produces(MyriaApiConstants.JSON_UTF_8) @Path("/dataset") @@ -76,6 +72,7 @@ public final class DatasetResource { /** The Myria server running on the master. */ @Context private Server server; + /** Information about the URL of the request. */ @Context private UriInfo uriInfo; @@ -186,10 +183,8 @@ public Response getDatasetData( /* Validate the request format. This will throw a MyriaApiException if format is invalid. */ String validFormat = validateFormat(format); - /* - * Allocate the pipes by which the {@link DataOutput} operator will talk to the {@link StreamingOutput} object that - * will stream data to the client. - */ + /* Allocate the pipes by which the {@link DataOutput} operator will talk to the {@link StreamingOutput} object that + * will stream data to the client. */ PipeSink dataSink = new PipeSink(); /* .. and make it the entity of the response. */ @@ -250,10 +245,8 @@ public Response getQueryData( /* Validate the request format. This will throw a MyriaApiException if format is invalid. */ String validFormat = validateFormat(format); - /* - * Allocate the pipes by which the {@link DataOutput} operator will talk to the {@link StreamingOutput} object that - * will stream data to the client. - */ + /* Allocate the pipes by which the {@link DataOutput} operator will talk to the {@link StreamingOutput} object that + * will stream data to the client. */ PipeSink dataSink = new PipeSink(); /* .. and make it the entity of the response. */ @@ -342,12 +335,10 @@ public Response replaceDataset( new TupleSource(new CsvTupleReader(schema, delimiter), new InputStreamSource(is)); ResponseBuilder builder = Response.ok(); - HowPartitioned howPartitioned = server.getDatasetStatus(relationKey).getHowPartitioned(); - PartitionFunction pf = howPartitioned.getPf(); - if (pf == null) { - pf = new RoundRobinPartitionFunction(null); - } - return doIngest(relationKey, source, howPartitioned.getWorkers(), null, true, builder, pf); + HowDistributed howDistributed = server.getDatasetStatus(relationKey).getHowDistributed(); + DistributeFunction df = + MoreObjects.firstNonNull(howDistributed.getDf(), new RoundRobinDistributeFunction()); + return doIngest(relationKey, source, howDistributed.getWorkers(), null, true, builder, df); } /** @@ -411,9 +402,7 @@ public Response persistDataset( return response.entity(queryId).build(); } - /** - * Creates an index based on the DbCreateIndexEncoding - */ + /** Creates an index based on the DbCreateIndexEncoding */ @POST @Path("/createIndex/") @Consumes(MediaType.APPLICATION_JSON) @@ -430,9 +419,7 @@ public Response createIndex(final CreateIndexEncoding encoding) throws DbExcepti return response.entity(queryId).build(); } - /** - * Creates an view based on the DbCreateViewEncoding - */ + /** Creates an view based on the DbCreateViewEncoding */ @POST @Path("/createView/") @Consumes(MediaType.APPLICATION_JSON) @@ -448,9 +435,7 @@ public Response createView(final CreateViewEncoding encoding) throws DbException return response.entity(queryId).build(); } - /** - * Creates an function based on DbCreateFunctionEncoding - */ + /** Creates an function based on DbCreateFunctionEncoding */ @POST @Path("/createFunction/") @Consumes(MediaType.APPLICATION_JSON) @@ -497,7 +482,7 @@ public Response newDataset(final DatasetEncoding dataset) throws DbException { dataset.indexes, dataset.overwrite, builder, - dataset.partitionFunction); + dataset.distributeFunction); } /** @@ -506,15 +491,15 @@ public Response newDataset(final DatasetEncoding dataset) throws DbException { * @param relationKey the name of the dataset to be ingested. * @param schema the {@link Schema} of the data. * @param binary optional: if true, indicates that supplied data should be interpreted as a packed binary - * object. (default, false) + * object. (default, false) * @param delimiter optional: the delimiter of the data, if a plaintext file is ingested. (binary is - * missing or false). + * missing or false). * @param isLittleEndian optional: indicates whether the data is in big-Endian (default, false) or - * little-Endian format. Only applicable when binary is set. + * little-Endian format. Only applicable when binary is set. * @param overwrite optional: indicates that an existing relation should be overwritten. If false, then a - * 409 Conflict response will be thrown if relationKey already exists in the catalog. + * 409 Conflict response will be thrown if relationKey already exists in the catalog. * @param data optional: the source of bytes to be loaded. - * @param partitionFunction optional: how the data will be partitioned, using RoundRobin by default. + * @param distributeFunction optional: how the data will be distributed, using RoundRobin by default. * @return the created dataset resource. * @throws DbException if there is an error in the database. */ @@ -528,7 +513,7 @@ public Response newDatasetMultipart( @FormDataParam("isLittleEndian") final Boolean isLittleEndian, @FormDataParam("overwrite") final Boolean overwrite, @FormDataParam("data") final InputStream data, - @FormDataParam("partitionFunction") final PartitionFunction partitionFunction) + @FormDataParam("distributeFunction") final DistributeFunction distributeFunction) throws DbException { Preconditions.checkArgument(relationKey != null, "Missing required field relationKey."); @@ -556,7 +541,7 @@ public Response newDatasetMultipart( null, overwrite, builder, - MoreObjects.firstNonNull(partitionFunction, new RoundRobinPartitionFunction(null))); + MoreObjects.firstNonNull(distributeFunction, new RoundRobinDistributeFunction())); } /** @@ -568,18 +553,18 @@ public Response newDatasetMultipart( * @param indexes any user-requested indexes to be created * @param overwrite whether an existing relation should be overwritten * @param builder the template response - * @param pf the partition function. + * @param df the distribute function * @return the created dataset resource * @throws DbException on any error */ private Response doIngest( final RelationKey relationKey, final Operator source, - final Set workers, + final List workers, final List> indexes, final Boolean overwrite, final ResponseBuilder builder, - @Nonnull final PartitionFunction pf) + @Nonnull final DistributeFunction df) throws DbException { /* Validate the workers that will ingest this dataset. */ @@ -597,7 +582,6 @@ private Response doIngest( Status.SERVICE_UNAVAILABLE, "Not all requested workers are alive"); } } - Set actualWorkers = MoreObjects.firstNonNull(workers, server.getAliveWorkers()); /* Check overwriting existing dataset. */ try { @@ -611,7 +595,7 @@ private Response doIngest( /* Do the ingest, blocking until complete. */ DatasetStatus status; try { - status = server.ingestDataset(relationKey, actualWorkers, indexes, source, pf); + status = server.ingestDataset(relationKey, workers, indexes, source, df); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return Response.status(Status.SERVICE_UNAVAILABLE).entity("Interrupted").build(); @@ -652,7 +636,7 @@ public Response parallelIngest(final ParallelDatasetEncoding dataset) dataset.numberOfSkippedLines, dataset.s3Source, dataset.workers, - dataset.partitionFunction); + dataset.distributeFunction); /* In the response, tell the client the path to the relation. */ URI datasetUri = getCanonicalResourcePath(uriInfo, dataset.relationKey); @@ -725,7 +709,7 @@ public Response newTipsyDataset( dataset.indexes, false, builder, - dataset.partitionFunction); + dataset.distributeFunction); } /** diff --git a/src/edu/washington/escience/myria/api/encoding/AbstractConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/AbstractConsumerEncoding.java index 6ff5a9520..940daec06 100644 --- a/src/edu/washington/escience/myria/api/encoding/AbstractConsumerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/AbstractConsumerEncoding.java @@ -6,8 +6,13 @@ import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.parallel.ExchangePairID; +/** + * @param type of the consumer + */ public abstract class AbstractConsumerEncoding extends LeafOperatorEncoding implements ExchangeEncoding { + + /** operator Id. */ @Required public Integer argOperatorId; Integer getArgOperatorId() { diff --git a/src/edu/washington/escience/myria/api/encoding/AbstractProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/AbstractProducerEncoding.java index acb6440bb..aac8b02d8 100644 --- a/src/edu/washington/escience/myria/api/encoding/AbstractProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/AbstractProducerEncoding.java @@ -6,6 +6,9 @@ import edu.washington.escience.myria.operator.network.Producer; import edu.washington.escience.myria.parallel.ExchangePairID; +/** + * @param

type of the producer. + */ public abstract class AbstractProducerEncoding

extends UnaryOperatorEncoding

implements ExchangeEncoding { private Set realWorkerIds; diff --git a/src/edu/washington/escience/myria/api/encoding/AggregateEncoding.java b/src/edu/washington/escience/myria/api/encoding/AggregateEncoding.java index 86c727dd5..a0782d680 100644 --- a/src/edu/washington/escience/myria/api/encoding/AggregateEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/AggregateEncoding.java @@ -4,7 +4,9 @@ import edu.washington.escience.myria.operator.agg.Aggregate; import edu.washington.escience.myria.operator.agg.AggregatorFactory; +/** JSON wrapper for Aggregate. */ public class AggregateEncoding extends UnaryOperatorEncoding { + /** aggregators. */ @Required public AggregatorFactory[] aggregators; @Override diff --git a/src/edu/washington/escience/myria/api/encoding/BroadcastConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/BroadcastConsumerEncoding.java index 5eb8a7f62..7d5446a6f 100644 --- a/src/edu/washington/escience/myria/api/encoding/BroadcastConsumerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/BroadcastConsumerEncoding.java @@ -1,21 +1,15 @@ package edu.washington.escience.myria.api.encoding; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; -import edu.washington.escience.myria.operator.network.GenericShuffleConsumer; +import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.util.MyriaUtils; -/** - * - * JSON wrapper for BroadcastConsumer - * - */ -public class BroadcastConsumerEncoding extends AbstractConsumerEncoding { +/** JSON wrapper for BroadcastConsumer */ +public class BroadcastConsumerEncoding extends AbstractConsumerEncoding { @Override - public GenericShuffleConsumer construct(ConstructArgs args) { - return new GenericShuffleConsumer( - null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - MyriaUtils.integerSetToIntArray(getRealWorkerIds())); + public Consumer construct(ConstructArgs args) { + return new Consumer( + null, MyriaUtils.getSingleElement(getRealOperatorIds()), getRealWorkerIds()); } } diff --git a/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java index 581f5e28f..eb28946b0 100644 --- a/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java @@ -2,29 +2,21 @@ import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; -import edu.washington.escience.myria.operator.network.partition.FixValuePartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; +import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.util.MyriaUtils; /** - * * JSON wrapper for BroadcastProducer - * */ public class BroadcastProducerEncoding extends AbstractProducerEncoding { @Override public GenericShuffleProducer construct(ConstructArgs args) { - int[][] cellPartition = new int[1][]; - int[] allCells = new int[getRealWorkerIds().size()]; - for (int i = 0; i < getRealWorkerIds().size(); i++) { - allCells[i] = i; - } - cellPartition[0] = allCells; return new GenericShuffleProducer( null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - cellPartition, + getRealOperatorIds().toArray(new ExchangePairID[] {}), MyriaUtils.integerSetToIntArray(getRealWorkerIds()), - new FixValuePartitionFunction(0)); + new BroadcastDistributeFunction(getRealWorkerIds().size())); } } diff --git a/src/edu/washington/escience/myria/api/encoding/CollectConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/CollectConsumerEncoding.java index 987ef91d9..e3986d89c 100644 --- a/src/edu/washington/escience/myria/api/encoding/CollectConsumerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/CollectConsumerEncoding.java @@ -1,16 +1,17 @@ package edu.washington.escience.myria.api.encoding; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; -import edu.washington.escience.myria.operator.network.CollectConsumer; +import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.util.MyriaUtils; -public class CollectConsumerEncoding extends AbstractConsumerEncoding { +/** + * JSON wrapper for CollectConsumer + */ +public class CollectConsumerEncoding extends AbstractConsumerEncoding { @Override - public CollectConsumer construct(ConstructArgs args) { - return new CollectConsumer( - null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - MyriaUtils.integerSetToIntArray(getRealWorkerIds())); + public Consumer construct(ConstructArgs args) { + return new Consumer( + null, MyriaUtils.getSingleElement(getRealOperatorIds()), getRealWorkerIds()); } } diff --git a/src/edu/washington/escience/myria/api/encoding/DatasetEncoding.java b/src/edu/washington/escience/myria/api/encoding/DatasetEncoding.java index 27582e845..f0179dc63 100644 --- a/src/edu/washington/escience/myria/api/encoding/DatasetEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/DatasetEncoding.java @@ -1,19 +1,18 @@ package edu.washington.escience.myria.api.encoding; import java.util.List; -import java.util.Set; import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.Schema; import edu.washington.escience.myria.accessmethod.AccessMethod.IndexRef; import edu.washington.escience.myria.io.DataSource; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; -import edu.washington.escience.myria.operator.network.partition.RoundRobinPartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.RoundRobinDistributeFunction; public class DatasetEncoding extends MyriaApiEncoding { @Required public RelationKey relationKey; @Required public Schema schema; - public Set workers; + public List workers; @Required public DataSource source; public Character delimiter; public Character escape; @@ -22,5 +21,5 @@ public class DatasetEncoding extends MyriaApiEncoding { public Boolean importFromDatabase; public List> indexes; public Boolean overwrite; - public PartitionFunction partitionFunction = new RoundRobinPartitionFunction(null); + public DistributeFunction distributeFunction = new RoundRobinDistributeFunction(); } diff --git a/src/edu/washington/escience/myria/api/encoding/DatasetStatus.java b/src/edu/washington/escience/myria/api/encoding/DatasetStatus.java index 0fa44973e..f16e140ea 100644 --- a/src/edu/washington/escience/myria/api/encoding/DatasetStatus.java +++ b/src/edu/washington/escience/myria/api/encoding/DatasetStatus.java @@ -7,23 +7,19 @@ import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.Schema; -import edu.washington.escience.myria.operator.network.partition.HowPartitioned; +import edu.washington.escience.myria.operator.network.distribute.HowDistributed; -/** - * Metadata about a dataset that has been loaded into the system. - */ +/** Metadata about a dataset that has been loaded into the system. */ public class DatasetStatus { - /** - * Instantiate a DatasetStatus with the provided values. + /** Instantiate a DatasetStatus with the provided values. * * @param relationKey The {@link RelationKey} identifying the dataset. * @param schema The {@link Schema} of the tuples in the dataset. * @param numTuples The number of tuples in the dataset. * @param queryId The query that created this dataset. * @param created When this dataset was created, in ISO8601 format. - * @param howPartitioned How this dataset was partitioned. - */ + * @param howDistributed How this dataset was distributed. */ @JsonCreator public DatasetStatus( @JsonProperty("relationKey") final RelationKey relationKey, @@ -31,13 +27,13 @@ public DatasetStatus( @JsonProperty("numTuples") final long numTuples, @JsonProperty("queryId") final long queryId, @JsonProperty("created") final String created, - @JsonProperty("howPartitioned") final HowPartitioned howPartitioned) { + @JsonProperty("howDistributed") final HowDistributed howDistributed) { this.relationKey = relationKey; this.schema = schema; this.numTuples = numTuples; this.queryId = queryId; this.created = created; - this.howPartitioned = howPartitioned; + this.howDistributed = howDistributed; } /** The {@link RelationKey} identifying the dataset. */ @@ -50,58 +46,44 @@ public DatasetStatus( @JsonProperty private final Long queryId; /** When this dataset was created, in ISO8601 format. */ @JsonProperty private final String created; - /** How this dataset was partitioned. */ - @JsonProperty private final HowPartitioned howPartitioned; + /** How this dataset was distributed. */ + @JsonProperty private final HowDistributed howDistributed; /** The URI of this resource. */ @JsonProperty public URI uri; - /** - * @return The {@link RelationKey} identifying the dataset. - */ + /** @return The {@link RelationKey} identifying the dataset. */ public RelationKey getRelationKey() { return relationKey; } - /** - * @return The {@link Schema} of the tuples in the dataset. - */ + /** @return The {@link Schema} of the tuples in the dataset. */ public Schema getSchema() { return schema; } - /** - * @return The number of tuples in the dataset. - */ + /** @return The number of tuples in the dataset. */ public long getNumTuples() { return numTuples; } - /** - * @return the queryId - */ + /** @return the queryId */ public Long getQueryId() { return queryId; } - /** - * @return the created - */ + /** @return the created */ public String getCreated() { return created; } - /** - * @return how the dataset was partitioned. - */ - public HowPartitioned getHowPartitioned() { - return howPartitioned; + /** @return how the dataset was partitioned. */ + public HowDistributed getHowDistributed() { + return howDistributed; } - /** - * Set the URI of this dataset. + /** Set the URI of this dataset. * - * @param datasetUri - */ + * @param datasetUri */ public void setUri(final URI datasetUri) { uri = datasetUri; } diff --git a/src/edu/washington/escience/myria/api/encoding/DbInsertEncoding.java b/src/edu/washington/escience/myria/api/encoding/DbInsertEncoding.java index 3f4d3e343..623316728 100644 --- a/src/edu/washington/escience/myria/api/encoding/DbInsertEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/DbInsertEncoding.java @@ -9,12 +9,9 @@ import edu.washington.escience.myria.accessmethod.ConnectionInfo; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.DbInsert; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; -/** - * A JSON-able wrapper for the expected wire message for a new dataset. - * - */ +/** A JSON-able wrapper for the expected wire message for a new dataset. */ public class DbInsertEncoding extends UnaryOperatorEncoding { /** The name under which the dataset will be stored. */ @Required public RelationKey relationKey; @@ -22,8 +19,8 @@ public class DbInsertEncoding extends UnaryOperatorEncoding { public Boolean argOverwriteTable; /** Indexes created. */ public List> indexes; - /** The PartitionFunction used to partition this relation. */ - public PartitionFunction partitionFunction; + /** The DistributeFunction used to distribute this relation. */ + public DistributeFunction distributeFunction; /** * The ConnectionInfo struct determines what database the data will be written to. If null, the worker's default @@ -36,6 +33,6 @@ public DbInsert construct(final ConstructArgs args) { /* default overwrite to {@code false}, so we append. */ argOverwriteTable = MoreObjects.firstNonNull(argOverwriteTable, Boolean.FALSE); return new DbInsert( - null, relationKey, connectionInfo, argOverwriteTable, indexes, partitionFunction); + null, relationKey, connectionInfo, argOverwriteTable, indexes, distributeFunction); } } diff --git a/src/edu/washington/escience/myria/api/encoding/GenericShuffleConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/GenericShuffleConsumerEncoding.java new file mode 100644 index 000000000..fb8452802 --- /dev/null +++ b/src/edu/washington/escience/myria/api/encoding/GenericShuffleConsumerEncoding.java @@ -0,0 +1,15 @@ +package edu.washington.escience.myria.api.encoding; + +import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; +import edu.washington.escience.myria.operator.network.Consumer; +import edu.washington.escience.myria.util.MyriaUtils; + +/** JSON wrapper for GenericShuffleConsumer. */ +public class GenericShuffleConsumerEncoding extends AbstractConsumerEncoding { + + @Override + public Consumer construct(ConstructArgs args) { + return new Consumer( + null, MyriaUtils.getSingleElement(getRealOperatorIds()), getRealWorkerIds()); + } +} diff --git a/src/edu/washington/escience/myria/api/encoding/ShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java similarity index 51% rename from src/edu/washington/escience/myria/api/encoding/ShuffleProducerEncoding.java rename to src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java index 61f1c5c99..cd23ca882 100644 --- a/src/edu/washington/escience/myria/api/encoding/ShuffleProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java @@ -1,30 +1,35 @@ package edu.washington.escience.myria.api.encoding; +import java.util.List; import java.util.Set; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.util.MyriaUtils; -/** - * A JSON-able wrapper for the expected wire message for a new dataset. - * - */ -public class ShuffleProducerEncoding extends AbstractProducerEncoding { - @Required public PartitionFunction argPf; +/** JSON wrapper for GenericShuffleProducer encoding. */ +public class GenericShuffleProducerEncoding + extends AbstractProducerEncoding { + + /** The distribute function. */ + @Required public DistributeFunction distributeFunction; + + /** Type of the buffer for recovery. */ public StreamingStateEncoding argBufferStateType; @Override public GenericShuffleProducer construct(final ConstructArgs args) { Set workerIds = getRealWorkerIds(); - argPf.setNumPartitions(workerIds.size()); + List operatorIds = getRealOperatorIds(); + distributeFunction.setNumDestinations(workerIds.size(), operatorIds.size()); GenericShuffleProducer producer = new GenericShuffleProducer( null, - MyriaUtils.getSingleElement(getRealOperatorIds()), + operatorIds.toArray(new ExchangePairID[] {}), MyriaUtils.integerSetToIntArray(workerIds), - argPf); + distributeFunction); if (argBufferStateType != null) { producer.setBackupBuffer(argBufferStateType); } diff --git a/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleConsumerEncoding.java new file mode 100644 index 000000000..e4359a31a --- /dev/null +++ b/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleConsumerEncoding.java @@ -0,0 +1,17 @@ +package edu.washington.escience.myria.api.encoding; + +import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; +import edu.washington.escience.myria.operator.network.Consumer; +import edu.washington.escience.myria.util.MyriaUtils; + +/** + * Consumer part of JSON encoding for HyperCube Join. + */ +public class HyperCubeShuffleConsumerEncoding extends AbstractConsumerEncoding { + + @Override + public Consumer construct(ConstructArgs args) { + return new Consumer( + null, MyriaUtils.getSingleElement(getRealOperatorIds()), getRealWorkerIds()); + } +} diff --git a/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java new file mode 100644 index 000000000..bc4457254 --- /dev/null +++ b/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java @@ -0,0 +1,44 @@ +package edu.washington.escience.myria.api.encoding; + +import java.util.Collections; +import java.util.List; + +import javax.ws.rs.core.Response.Status; + +import edu.washington.escience.myria.api.MyriaApiException; +import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; +import edu.washington.escience.myria.operator.network.GenericShuffleProducer; +import edu.washington.escience.myria.operator.network.distribute.HyperCubeDistributeFunction; +import edu.washington.escience.myria.parallel.ExchangePairID; +import edu.washington.escience.myria.util.MyriaUtils; + +/** + * JSON Encoding for HyperCubeShuffle. + */ +public class HyperCubeShuffleProducerEncoding + extends AbstractProducerEncoding { + + /** distribute function. */ + @Required public HyperCubeDistributeFunction distributeFunction; + + @Override + public GenericShuffleProducer construct(ConstructArgs args) throws MyriaApiException { + return new GenericShuffleProducer( + null, + getRealOperatorIds().toArray(new ExchangePairID[] {}), + MyriaUtils.integerSetToIntArray( + args.getServer().getRandomWorkers(distributeFunction.getAllDestinations().size())), + distributeFunction); + } + + @Override + protected void validateExtra() { + List values = distributeFunction.getAllDestinations(); + Collections.sort(values); + for (int i = 0; i < values.size(); i++) { + if (values.get(i) != i) { + throw new MyriaApiException(Status.BAD_REQUEST, "invalid cell partition"); + } + } + } +} diff --git a/src/edu/washington/escience/myria/api/encoding/HyperShuffleConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/HyperShuffleConsumerEncoding.java deleted file mode 100644 index 10ae8050a..000000000 --- a/src/edu/washington/escience/myria/api/encoding/HyperShuffleConsumerEncoding.java +++ /dev/null @@ -1,20 +0,0 @@ -package edu.washington.escience.myria.api.encoding; - -import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; -import edu.washington.escience.myria.operator.network.GenericShuffleConsumer; -import edu.washington.escience.myria.util.MyriaUtils; - -/** - * Consumer part of JSON encoding for HyperCube Join. - * - */ -public class HyperShuffleConsumerEncoding extends AbstractConsumerEncoding { - - @Override - public GenericShuffleConsumer construct(ConstructArgs args) { - return new GenericShuffleConsumer( - null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - MyriaUtils.integerSetToIntArray(getRealWorkerIds())); - } -} diff --git a/src/edu/washington/escience/myria/api/encoding/HyperShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/HyperShuffleProducerEncoding.java deleted file mode 100644 index fc5d1aee6..000000000 --- a/src/edu/washington/escience/myria/api/encoding/HyperShuffleProducerEncoding.java +++ /dev/null @@ -1,65 +0,0 @@ -package edu.washington.escience.myria.api.encoding; - -import javax.ws.rs.core.Response.Status; - -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.api.MyriaApiException; -import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; -import edu.washington.escience.myria.operator.network.GenericShuffleProducer; -import edu.washington.escience.myria.operator.network.partition.MFMDHashPartitionFunction; -import edu.washington.escience.myria.util.MyriaArrayUtils; -import edu.washington.escience.myria.util.MyriaUtils; - -/** - * Producer part of JSON Encoding for HyperCube Join. - * - */ -public class HyperShuffleProducerEncoding extends AbstractProducerEncoding { - - @Required public int[] hashedColumns; - @Required public int[] mappedHCDimensions; - @Required public int[] hyperCubeDimensions; - @Required public int[][] cellPartition; - - @Override - public GenericShuffleProducer construct(ConstructArgs args) throws MyriaApiException { - - /* - * Validate whether number of workers matches cube dimensions. - * - * has to validate here because until now the workers has been set. - */ - int numCells = 1; - for (int d : hyperCubeDimensions) { - numCells = numCells * d; - } - for (int[] partition : cellPartition) { - for (int cellId : partition) { - Preconditions.checkElementIndex(cellId, getRealWorkerIds().size()); - } - } - - /* constructing a MFMDHashPartitionFunction. */ - MFMDHashPartitionFunction pf = - new MFMDHashPartitionFunction( - cellPartition.length, hyperCubeDimensions, hashedColumns, mappedHCDimensions); - - return new GenericShuffleProducer( - null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - cellPartition, - MyriaUtils.integerSetToIntArray(args.getServer().getRandomWorkers(numCells)), - pf); - } - - @Override - protected void validateExtra() { - int[] arr = MyriaArrayUtils.arrayFlattenThenSort(cellPartition); - for (int i = 0; i < arr.length; i++) { - if (arr[i] != i) { - throw new MyriaApiException(Status.BAD_REQUEST, "invalid cell partition"); - } - } - } -} diff --git a/src/edu/washington/escience/myria/api/encoding/LocalMultiwayConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/LocalMultiwayConsumerEncoding.java index 120b2f14e..731dce0ff 100644 --- a/src/edu/washington/escience/myria/api/encoding/LocalMultiwayConsumerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/LocalMultiwayConsumerEncoding.java @@ -1,17 +1,20 @@ package edu.washington.escience.myria.api.encoding; +import com.google.common.collect.ImmutableSet; + import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; -import edu.washington.escience.myria.operator.network.LocalMultiwayConsumer; +import edu.washington.escience.myria.operator.network.Consumer; +import edu.washington.escience.myria.parallel.ipc.IPCConnectionPool; import edu.washington.escience.myria.util.MyriaUtils; -/** - * A JSON-able wrapper for the expected wire message for a new dataset. - * - */ -public class LocalMultiwayConsumerEncoding extends AbstractConsumerEncoding { +/** A JSON-able wrapper for the expected wire message for a new dataset. */ +public class LocalMultiwayConsumerEncoding extends AbstractConsumerEncoding { @Override - public LocalMultiwayConsumer construct(ConstructArgs args) { - return new LocalMultiwayConsumer(null, MyriaUtils.getSingleElement(getRealOperatorIds())); + public Consumer construct(ConstructArgs args) { + return new Consumer( + null, + MyriaUtils.getSingleElement(getRealOperatorIds()), + ImmutableSet.of(IPCConnectionPool.SELF_IPC_ID)); } } diff --git a/src/edu/washington/escience/myria/api/encoding/LocalMultiwayProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/LocalMultiwayProducerEncoding.java index aed663214..2aff7d606 100644 --- a/src/edu/washington/escience/myria/api/encoding/LocalMultiwayProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/LocalMultiwayProducerEncoding.java @@ -1,20 +1,14 @@ package edu.washington.escience.myria.api.encoding; -import java.util.List; - import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.LocalMultiwayProducer; import edu.washington.escience.myria.parallel.ExchangePairID; -/** - * A JSON-able wrapper for the expected wire message for a new dataset. - * - */ +/** JSON wrapper for LocalMultiwayProducer. */ public class LocalMultiwayProducerEncoding extends AbstractProducerEncoding { @Override public LocalMultiwayProducer construct(ConstructArgs args) { - List ids = getRealOperatorIds(); - return new LocalMultiwayProducer(null, ids.toArray(new ExchangePairID[ids.size()])); + return new LocalMultiwayProducer(null, getRealOperatorIds().toArray(new ExchangePairID[] {})); } } diff --git a/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java b/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java index 525c129d6..6ee54c962 100644 --- a/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java @@ -12,13 +12,9 @@ import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.Operator; -/** - * A JSON-able wrapper for the expected wire message for an operator. To add a new operator, two things need to be done. - * - * 1. Create an Encoding class that extends OperatorEncoding. - * - * 2. Add the operator to the list of (alphabetically sorted) JsonSubTypes below. - */ +/** A JSON-able wrapper for the expected wire message for an operator. To add a new operator, two things need to be + * done. 1. Create an Encoding class that extends OperatorEncoding. 2. Add the operator to the list of (alphabetically + * sorted) JsonSubTypes below. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "opType") @JsonSubTypes({ @Type(name = "Aggregate", value = AggregateEncoding.class), @@ -42,8 +38,8 @@ @Type(name = "EmptySink", value = EmptySinkEncoding.class), @Type(name = "EOSController", value = EOSControllerEncoding.class), @Type(name = "Filter", value = FilterEncoding.class), - @Type(name = "HyperShuffleProducer", value = HyperShuffleProducerEncoding.class), - @Type(name = "HyperShuffleConsumer", value = HyperShuffleConsumerEncoding.class), + @Type(name = "HyperCubeShuffleProducer", value = HyperCubeShuffleProducerEncoding.class), + @Type(name = "HyperCubeShuffleConsumer", value = HyperCubeShuffleConsumerEncoding.class), @Type(name = "IDBController", value = IDBControllerEncoding.class), @Type(name = "InMemoryOrderBy", value = InMemoryOrderByEncoding.class), @Type(name = "LeapFrogJoin", value = LeapFrogJoinEncoding.class), @@ -61,8 +57,8 @@ @Type(name = "SamplingDistribution", value = SamplingDistributionEncoding.class), @Type(name = "SeaFlowScan", value = SeaFlowFileScanEncoding.class), @Type(name = "SetGlobal", value = SetGlobalEncoding.class), - @Type(name = "ShuffleConsumer", value = ShuffleConsumerEncoding.class), - @Type(name = "ShuffleProducer", value = ShuffleProducerEncoding.class), + @Type(name = "ShuffleConsumer", value = GenericShuffleConsumerEncoding.class), + @Type(name = "ShuffleProducer", value = GenericShuffleProducerEncoding.class), @Type(name = "SingleGroupByAggregate", value = SingleGroupByAggregateEncoding.class), @Type(name = "Singleton", value = SingletonEncoding.class), @Type(name = "StatefulApply", value = StatefulApplyEncoding.class), @@ -82,14 +78,10 @@ public abstract class OperatorEncoding extends MyriaApiEncod public String opName; - /** - * Connect any operators to this one. - */ + /** Connect any operators to this one. */ public abstract void connect(Operator operator, Map operators); - /** - * @param args TODO - * @return an instantiated operator. - */ + /** @param args TODO + * @return an instantiated operator. */ public abstract T construct(@Nonnull ConstructArgs args) throws MyriaApiException; } diff --git a/src/edu/washington/escience/myria/api/encoding/ParallelDatasetEncoding.java b/src/edu/washington/escience/myria/api/encoding/ParallelDatasetEncoding.java index 989f3dc6b..a231c11c6 100644 --- a/src/edu/washington/escience/myria/api/encoding/ParallelDatasetEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/ParallelDatasetEncoding.java @@ -8,8 +8,8 @@ import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.Schema; import edu.washington.escience.myria.io.AmazonS3Source; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; -import edu.washington.escience.myria.operator.network.partition.RoundRobinPartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.RoundRobinDistributeFunction; /** * @@ -23,5 +23,5 @@ public class ParallelDatasetEncoding extends MyriaApiEncoding { public Integer numberOfSkippedLines; public Character quote; public Set workers; - public PartitionFunction partitionFunction = new RoundRobinPartitionFunction(null); + public DistributeFunction distributeFunction = new RoundRobinDistributeFunction(); } diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index 75530cf62..97c09d123 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -40,7 +40,6 @@ import edu.washington.escience.myria.operator.agg.MultiGroupByAggregate; import edu.washington.escience.myria.operator.agg.PrimitiveAggregator.AggregationOp; import edu.washington.escience.myria.operator.agg.SingleColumnAggregatorFactory; -import edu.washington.escience.myria.operator.network.CollectConsumer; import edu.washington.escience.myria.operator.network.CollectProducer; import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.operator.network.EOSController; @@ -677,7 +676,7 @@ public static SubQuery getRelationTupleUpdateSubQuery( } /* Master plan: collect, sum, insert the updates. */ - CollectConsumer consumer = new CollectConsumer(schema, collectId, workerPlans.keySet()); + Consumer consumer = new Consumer(schema, collectId, workerPlans.keySet()); MultiGroupByAggregate aggCounts = new MultiGroupByAggregate( consumer, new int[] {0, 1, 2}, new SingleColumnAggregatorFactory(3, AggregationOp.SUM)); diff --git a/src/edu/washington/escience/myria/api/encoding/ShuffleConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/ShuffleConsumerEncoding.java deleted file mode 100644 index 596cc7ccd..000000000 --- a/src/edu/washington/escience/myria/api/encoding/ShuffleConsumerEncoding.java +++ /dev/null @@ -1,20 +0,0 @@ -package edu.washington.escience.myria.api.encoding; - -import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; -import edu.washington.escience.myria.operator.network.GenericShuffleConsumer; -import edu.washington.escience.myria.util.MyriaUtils; - -/** - * A JSON-able wrapper for the expected wire message for a new dataset. - * - */ -public class ShuffleConsumerEncoding extends AbstractConsumerEncoding { - - @Override - public GenericShuffleConsumer construct(ConstructArgs args) { - return new GenericShuffleConsumer( - null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - MyriaUtils.integerSetToIntArray(getRealWorkerIds())); - } -} diff --git a/src/edu/washington/escience/myria/api/encoding/TempInsertEncoding.java b/src/edu/washington/escience/myria/api/encoding/TempInsertEncoding.java index bf9b92fd6..da3243ef8 100644 --- a/src/edu/washington/escience/myria/api/encoding/TempInsertEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/TempInsertEncoding.java @@ -9,12 +9,9 @@ import edu.washington.escience.myria.accessmethod.ConnectionInfo; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.DbInsertTemp; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; -/** - * A JSON-able wrapper for the expected wire message for a new dataset. - * - */ +/** A JSON-able wrapper for the expected wire message for a new dataset. */ public class TempInsertEncoding extends UnaryOperatorEncoding { /** The name under which the dataset will be stored. */ @Required public String table; @@ -22,13 +19,11 @@ public class TempInsertEncoding extends UnaryOperatorEncoding { public Boolean argOverwriteTable; /** Indexes created. */ public List> indexes; - /** The PartitionFunction used to partition this relation. */ - public PartitionFunction partitionFunction; + /** The DistributeFunction used to distribute this relation. */ + public DistributeFunction distributeFunction; - /** - * The ConnectionInfo struct determines what database the data will be written to. If null, the worker's default - * database will be used. - */ + /** The ConnectionInfo struct determines what database the data will be written to. If null, the worker's default + * database will be used. */ public ConnectionInfo connectionInfo; @Override @@ -41,6 +36,6 @@ public DbInsertTemp construct(final ConstructArgs args) { connectionInfo, argOverwriteTable, indexes, - partitionFunction); + distributeFunction); } } diff --git a/src/edu/washington/escience/myria/api/encoding/TipsyDatasetEncoding.java b/src/edu/washington/escience/myria/api/encoding/TipsyDatasetEncoding.java index 99f28b5d4..cb2d54f9e 100644 --- a/src/edu/washington/escience/myria/api/encoding/TipsyDatasetEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/TipsyDatasetEncoding.java @@ -1,15 +1,14 @@ package edu.washington.escience.myria.api.encoding; import java.util.List; -import java.util.Set; import javax.ws.rs.core.Response.Status; import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.accessmethod.AccessMethod.IndexRef; import edu.washington.escience.myria.api.MyriaApiException; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; -import edu.washington.escience.myria.operator.network.partition.RoundRobinPartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.RoundRobinDistributeFunction; import edu.washington.escience.myria.util.FSUtils; public class TipsyDatasetEncoding extends MyriaApiEncoding { @@ -17,16 +16,14 @@ public class TipsyDatasetEncoding extends MyriaApiEncoding { @Required public String tipsyFilename; @Required public String grpFilename; @Required public String iorderFilename; - public Set workers; + public List workers; public List> indexes; - public PartitionFunction partitionFunction = new RoundRobinPartitionFunction(null); + public DistributeFunction distributeFunction = new RoundRobinDistributeFunction(); @Override public void validateExtra() { - /* - * Note we can only do this because we know that the operator will be run on the master. So we can't do this e.g. - * for TipsyFileScan because that might be run on a worker. - */ + /* Note we can only do this because we know that the operator will be run on the master. So we can't do this e.g. + * for TipsyFileScan because that might be run on a worker. */ try { FSUtils.checkFileReadable(tipsyFilename); FSUtils.checkFileReadable(grpFilename); diff --git a/src/edu/washington/escience/myria/coordinator/MasterCatalog.java b/src/edu/washington/escience/myria/coordinator/MasterCatalog.java index 493887482..daf548f01 100644 --- a/src/edu/washington/escience/myria/coordinator/MasterCatalog.java +++ b/src/edu/washington/escience/myria/coordinator/MasterCatalog.java @@ -50,7 +50,7 @@ import edu.washington.escience.myria.api.encoding.QueryEncoding; import edu.washington.escience.myria.api.encoding.QueryStatusEncoding; import edu.washington.escience.myria.api.encoding.plan.SubPlanEncoding; -import edu.washington.escience.myria.operator.network.partition.HowPartitioned; +import edu.washington.escience.myria.operator.network.distribute.HowDistributed; import edu.washington.escience.myria.parallel.Query; import edu.washington.escience.myria.parallel.RelationWriteMetadata; import edu.washington.escience.myria.parallel.SubQueryId; @@ -129,7 +129,7 @@ public final class MasterCatalog { + " program_name TEXT NOT NULL,\n" + " relation_name TEXT NOT NULL,\n" + " num_shards INTEGER NOT NULL,\n" - + " how_partitioned TEXT NOT NULL,\n" + + " how_distributed TEXT NOT NULL,\n" + " FOREIGN KEY (user_name,program_name,relation_name) REFERENCES relations ON DELETE CASCADE);"; /** Create an index on the stored_relations table. */ private static final String CREATE_STORED_RELATIONS_INDEX = @@ -419,26 +419,26 @@ private void addRelationMetadata( * @param sqliteConnection the connection to the SQLite database. * @param relation the relation to create. * @param workers the IDs of the workers storing this copy of the relation. - * @param howPartitioned how this copy of the relation is partitioned. + * @param howDistributed how this copy of the relation is distributed. * @throws CatalogException if there is an error in the database. */ private void addStoredRelation( final SQLiteConnection sqliteConnection, final RelationKey relation, final Set workers, - final HowPartitioned howPartitioned) + final HowDistributed howDistributed) throws CatalogException { try { /* First, populate the stored_relation table. */ SQLiteStatement statement = sqliteConnection.prepare( - "INSERT INTO stored_relations (user_name,program_name,relation_name,num_shards,how_partitioned) VALUES (?,?,?,?,?);"); + "INSERT INTO stored_relations (user_name,program_name,relation_name,num_shards,how_distributed) VALUES (?,?,?,?,?);"); statement.bind(1, relation.getUserName()); statement.bind(2, relation.getProgramName()); statement.bind(3, relation.getRelationName()); statement.bind(4, workers.size()); try { - statement.bind(5, MyriaJsonMapperProvider.getMapper().writeValueAsString(howPartitioned)); + statement.bind(5, MyriaJsonMapperProvider.getMapper().writeValueAsString(howDistributed)); } catch (JsonProcessingException e) { throw new CatalogException(e); } @@ -565,7 +565,7 @@ protected List job(final SQLiteConnection sqliteConnection) try { SQLiteStatement statement = sqliteConnection.prepare( - "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_partitioned FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); + "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_distributed FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); return datasetStatusListHelper(statement, sqliteConnection); } catch (final SQLiteException e) { throw new CatalogException(e); @@ -599,7 +599,7 @@ protected List job(final SQLiteConnection sqliteConnection) try { SQLiteStatement statement = sqliteConnection.prepare( - "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_partitioned FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE user_name=? AND is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); + "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_distributed FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE user_name=? AND is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); statement.bind(1, userName); return datasetStatusListHelper(statement, sqliteConnection); } catch (final SQLiteException e) { @@ -636,7 +636,7 @@ protected List job(final SQLiteConnection sqliteConnection) try { SQLiteStatement statement = sqliteConnection.prepare( - "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_partitioned FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE user_name=? AND program_name=? AND is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); + "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_distributed FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE user_name=? AND program_name=? AND is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); statement.bind(1, userName); statement.bind(2, programName); return datasetStatusListHelper(statement, sqliteConnection); @@ -672,7 +672,7 @@ protected List job(final SQLiteConnection sqliteConnection) try { SQLiteStatement statement = sqliteConnection.prepare( - "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_partitioned FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE query_id=? AND is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); + "SELECT user_name, program_name, relation_name, num_tuples, query_id, finish_time, how_distributed FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE query_id=? AND is_deleted=0 ORDER BY user_name, program_name, relation_name ASC"); statement.bind(1, queryId); return datasetStatusListHelper(statement, sqliteConnection); } catch (final SQLiteException e) { @@ -705,14 +705,14 @@ private static List datasetStatusListHelper( long numTuples = statement.columnLong(3); long queryId = statement.columnLong(4); String created = statement.columnString(5); - HowPartitioned howPartitioned; + HowDistributed howDistributed; try { - howPartitioned = + howDistributed = MyriaJsonMapperProvider.getMapper() - .readValue(statement.columnString(6), HowPartitioned.class); + .readValue(statement.columnString(6), HowDistributed.class); } catch (final IOException e) { - LOGGER.debug("Error deserializing howPartitioned for dataset #{}", relationKey, e); - howPartitioned = new HowPartitioned(null, null); + LOGGER.debug("Error deserializing howDistributed for dataset #{}", relationKey, e); + howDistributed = new HowDistributed(null, null); } result.add( new DatasetStatus( @@ -721,7 +721,7 @@ private static List datasetStatusListHelper( numTuples, queryId, created, - howPartitioned)); + howDistributed)); } statement.dispose(); return result.build(); @@ -1249,10 +1249,10 @@ protected Object job(final SQLiteConnection sqliteConnection) * Update the partition function of an ingested dataset. * * @param key the relation key. - * @param howPartitioned how the dataset was partitioned. + * @param howDistributed how the dataset was distributed. * @throws CatalogException if there is an error in the catalog. */ - public void updateHowPartitioned(final RelationKey key, final HowPartitioned howPartitioned) + public void updateHowPartitioned(final RelationKey key, final HowDistributed howDistributed) throws CatalogException { if (isClosed) { throw new CatalogException("Catalog is closed."); @@ -1268,11 +1268,11 @@ protected DatasetStatus job(final SQLiteConnection sqliteConnection) try { SQLiteStatement statement = sqliteConnection.prepare( - "UPDATE stored_relations set how_partitioned=? WHERE user_name=? AND program_name=? AND relation_name=?"); + "UPDATE stored_relations set how_distributed=? WHERE user_name=? AND program_name=? AND relation_name=?"); try { statement.bind( 1, - MyriaJsonMapperProvider.getMapper().writeValueAsString(howPartitioned)); + MyriaJsonMapperProvider.getMapper().writeValueAsString(howDistributed)); } catch (JsonProcessingException e) { throw new CatalogException(e); } @@ -1317,7 +1317,7 @@ protected DatasetStatus job(final SQLiteConnection sqliteConnection) try { SQLiteStatement statement = sqliteConnection.prepare( - "SELECT num_tuples, query_id, finish_time, how_partitioned FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE user_name=? AND program_name=? AND relation_name=?"); + "SELECT num_tuples, query_id, finish_time, how_distributed FROM relations JOIN queries USING (query_id) JOIN stored_relations USING (user_name,program_name,relation_name) WHERE user_name=? AND program_name=? AND relation_name=?"); statement.bind(1, relationKey.getUserName()); statement.bind(2, relationKey.getProgramName()); statement.bind(3, relationKey.getRelationName()); @@ -1328,17 +1328,17 @@ protected DatasetStatus job(final SQLiteConnection sqliteConnection) long numTuples = statement.columnLong(0); long queryId = statement.columnLong(1); String created = statement.columnString(2); - HowPartitioned howPartitioned; + HowDistributed howPartitioned; try { howPartitioned = MyriaJsonMapperProvider.getMapper() - .readValue(statement.columnString(3), HowPartitioned.class); + .readValue(statement.columnString(3), HowDistributed.class); } catch (final IOException e) { LOGGER.debug( "Error deserializing howPartitioned for dataset #{}", relationKey.toString(), e); - howPartitioned = new HowPartitioned(null, null); + howPartitioned = new HowDistributed(null, null); } statement.dispose(); return new DatasetStatus( @@ -1796,7 +1796,8 @@ protected Void job(final SQLiteConnection sqliteConnection) sqliteConnection, relation, workers, - new HowPartitioned(meta.getPartitionFunction(), Ints.toArray(workers))); + new HowDistributed( + meta.getDistributeFunction(), Ints.toArray(workers))); LOGGER.debug( "SubQuery #{} - adding {} to store shard of {}", subQueryId, diff --git a/src/edu/washington/escience/myria/operator/DbInsert.java b/src/edu/washington/escience/myria/operator/DbInsert.java index c980cd2fd..033f738b2 100644 --- a/src/edu/washington/escience/myria/operator/DbInsert.java +++ b/src/edu/washington/escience/myria/operator/DbInsert.java @@ -20,16 +20,13 @@ import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.accessmethod.AccessMethod; import edu.washington.escience.myria.accessmethod.AccessMethod.IndexRef; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.accessmethod.ConnectionInfo; import edu.washington.escience.myria.accessmethod.SQLiteInfo; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; import edu.washington.escience.myria.parallel.RelationWriteMetadata; import edu.washington.escience.myria.storage.TupleBatch; -/** - * @author valmeida - * - */ +/** @author valmeida */ public class DbInsert extends AbstractDbInsert { /** Required for Java serialization. */ @@ -46,47 +43,41 @@ public class DbInsert extends AbstractDbInsert { private RelationKey tempRelationKey; /** The indexes to be created on the table. Each entry is a list of columns. */ private final List> indexes; - /** The PartitionFunction used to partition the table across workers. */ - private final PartitionFunction partitionFunction; + /** The DistributeFunction used to distribute the table across workers. */ + private final DistributeFunction distributeFunction; - /** - * Constructs an insertion operator to store the tuples from the specified child into the specified database. If the + /** Constructs an insertion operator to store the tuples from the specified child into the specified database. If the * table does not exist, it will be created; if it does exist then old data will persist and new data will be * inserted. * * @param child the source of tuples to be inserted. * @param relationKey the key of the table the tuples should be inserted into. - * @param connectionInfo the parameters of the database connection. - */ + * @param connectionInfo the parameters of the database connection. */ public DbInsert( final Operator child, final RelationKey relationKey, final ConnectionInfo connectionInfo) { this(child, relationKey, connectionInfo, false); } - /** - * Constructs an insertion operator to store the tuples from the specified child into the worker's default database. + /** Constructs an insertion operator to store the tuples from the specified child into the worker's default database. * If the table does not exist, it will be created. If overwriteTable is true, any existing * data will be dropped. * * @param child the source of tuples to be inserted. * @param relationKey the key of the table the tuples should be inserted into. - * @param overwriteTable whether to overwrite a table that already exists. - */ + * @param overwriteTable whether to overwrite a table that already exists. */ public DbInsert( final Operator child, final RelationKey relationKey, final boolean overwriteTable) { this(child, relationKey, null, overwriteTable); } - /** - * Constructs an insertion operator to store the tuples from the specified child into the specified database. If the + /** Constructs an insertion operator to store the tuples from the specified child into the specified database. If the * table does not exist, it will be created. If overwriteTable is true, any existing data * will be dropped. * * @param child the source of tuples to be inserted. * @param relationKey the key of the table the tuples should be inserted into. * @param overwriteTable whether to overwrite a table that already exists. - * @param indexes indexes created. - */ + * @param indexes indexes created. */ public DbInsert( final Operator child, final RelationKey relationKey, @@ -95,16 +86,14 @@ public DbInsert( this(child, relationKey, null, overwriteTable, indexes); } - /** - * Constructs an insertion operator to store the tuples from the specified child into the specified database. If the + /** Constructs an insertion operator to store the tuples from the specified child into the specified database. If the * table does not exist, it will be created. If overwriteTable is true, any existing data * will be dropped. * * @param child the source of tuples to be inserted. * @param relationKey the key of the table the tuples should be inserted into. * @param connectionInfo the parameters of the database connection. - * @param overwriteTable whether to overwrite a table that already exists. - */ + * @param overwriteTable whether to overwrite a table that already exists. */ public DbInsert( final Operator child, final RelationKey relationKey, @@ -113,8 +102,7 @@ public DbInsert( this(child, relationKey, connectionInfo, overwriteTable, null); } - /** - * Constructs an insertion operator to store the tuples from the specified child into the specified database. If the + /** Constructs an insertion operator to store the tuples from the specified child into the specified database. If the * table does not exist, it will be created. If overwriteTable is true, any existing data * will be dropped. * @@ -122,8 +110,7 @@ public DbInsert( * @param relationKey the key of the table the tuples should be inserted into. * @param connectionInfo the parameters of the database connection. * @param overwriteTable whether to overwrite a table that already exists. - * @param indexes the indexes to be created on the table. Each entry is a list of columns. - */ + * @param indexes the indexes to be created on the table. Each entry is a list of columns. */ public DbInsert( final Operator child, final RelationKey relationKey, @@ -133,8 +120,7 @@ public DbInsert( this(child, relationKey, connectionInfo, overwriteTable, indexes, null); } - /** - * Constructs an insertion operator to store the tuples from the specified child into the specified database. If the + /** Constructs an insertion operator to store the tuples from the specified child into the specified database. If the * table does not exist, it will be created. If overwriteTable is true, any existing data * will be dropped. * @@ -143,34 +129,26 @@ public DbInsert( * @param connectionInfo the parameters of the database connection. * @param overwriteTable whether to overwrite a table that already exists. * @param indexes the indexes to be created on the table. Each entry is a list of columns. - * @param partitionFunction the PartitionFunction used to partition the table across workers. - */ + * @param partitionFunction the PartitionFunction used to partition the table across workers. */ public DbInsert( final Operator child, final RelationKey relationKey, final ConnectionInfo connectionInfo, final boolean overwriteTable, final List> indexes, - final PartitionFunction partitionFunction) { + final DistributeFunction distributeFunction) { super(child); Objects.requireNonNull(relationKey, "relationKey"); this.connectionInfo = connectionInfo; this.relationKey = relationKey; this.overwriteTable = overwriteTable; - this.partitionFunction = partitionFunction; + this.distributeFunction = distributeFunction; /* Sanity check arguments -- cannot create an index in append mode. */ Preconditions.checkArgument( overwriteTable || indexes == null || indexes.size() == 0, "Cannot create indexes when appending to a relation."); - /* - * 1) construct immutable copies of the given indexes. - * - * 2) ensure that the index requests are valid: - * - * - lists of column references must be non-null. - * - * - column references are unique per index. - */ + /* 1) construct immutable copies of the given indexes. 2) ensure that the index requests are valid: - lists of + * column references must be non-null. - column references are unique per index. */ if (indexes != null) { ImmutableList.Builder> index = ImmutableList.builder(); for (List i : indexes) { @@ -267,9 +245,7 @@ protected void childEOS() throws DbException { @Override protected void childEOI() throws DbException {} - /** - * @return the name of the relation that this operator will write to. - */ + /** @return the name of the relation that this operator will write to. */ public RelationKey getRelationKey() { return relationKey; } @@ -279,6 +255,6 @@ public Map writeSet() { return ImmutableMap.of( relationKey, new RelationWriteMetadata( - relationKey, getSchema(), overwriteTable, false, partitionFunction)); + relationKey, getSchema(), overwriteTable, false, distributeFunction)); } } diff --git a/src/edu/washington/escience/myria/operator/DbInsertTemp.java b/src/edu/washington/escience/myria/operator/DbInsertTemp.java index 5648a7963..10ead4f2d 100644 --- a/src/edu/washington/escience/myria/operator/DbInsertTemp.java +++ b/src/edu/washington/escience/myria/operator/DbInsertTemp.java @@ -20,15 +20,13 @@ import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.accessmethod.AccessMethod; import edu.washington.escience.myria.accessmethod.AccessMethod.IndexRef; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.accessmethod.ConnectionInfo; import edu.washington.escience.myria.accessmethod.SQLiteInfo; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; import edu.washington.escience.myria.parallel.RelationWriteMetadata; import edu.washington.escience.myria.storage.TupleBatch; -/** - * A temporary relation that is inserted into the database. - */ +/** A temporary relation that is inserted into the database. */ public class DbInsertTemp extends AbstractDbInsert { /** Required for Java serialization. */ @@ -45,20 +43,18 @@ public class DbInsertTemp extends AbstractDbInsert { private final boolean overwriteTable; /** The indexes to be created on the table. Each entry is a list of columns. */ private final List> indexes; - /** The PartitionFunction used to partition the table across workers. */ - private final PartitionFunction partitionFunction; + /** The DistributeFunction used to distribute the table across workers. */ + private final DistributeFunction distributeFunction; - /** - * Constructs a temporary insertion operator to store the tuples from the specified child into the specified relation. - * If the table does not exist, it will be created. If overwriteTable is true, any existing - * data will be dropped. + /** Constructs a temporary insertion operator to store the tuples from the specified child into the specified + * relation. If the table does not exist, it will be created. If overwriteTable is true, any + * existing data will be dropped. * * @param child the source of tuples to be inserted. * @param relationKey the key of the table the tuples should be inserted into. * @param connectionInfo the parameters of the database connection. * @param overwriteTable whether to overwrite a table that already exists. - * @param indexes the indexes to be created on the table. Each entry is a list of columns. - */ + * @param indexes the indexes to be created on the table. Each entry is a list of columns. */ public DbInsertTemp( final Operator child, final RelationKey relationKey, @@ -68,44 +64,35 @@ public DbInsertTemp( this(child, relationKey, connectionInfo, overwriteTable, indexes, null); } - /** - * Constructs a temporary insertion operator to store the tuples from the specified child into the specified relation. - * If the table does not exist, it will be created. If overwriteTable is true, any existing - * data will be dropped. + /** Constructs a temporary insertion operator to store the tuples from the specified child into the specified + * relation. If the table does not exist, it will be created. If overwriteTable is true, any + * existing data will be dropped. * * @param child the source of tuples to be inserted. * @param relationKey the key of the table the tuples should be inserted into. * @param connectionInfo the parameters of the database connection. * @param overwriteTable whether to overwrite a table that already exists. * @param indexes the indexes to be created on the table. Each entry is a list of columns. - * @param partitionFunction the PartitionFunction used to partition the table across workers. - */ + * @param distributeFunction the DistributeFunction used to distribute the table across workers. */ public DbInsertTemp( final Operator child, final RelationKey relationKey, final ConnectionInfo connectionInfo, final boolean overwriteTable, final List> indexes, - final PartitionFunction partitionFunction) { + final DistributeFunction distributeFunction) { super(child); Objects.requireNonNull(relationKey, "relationKey"); this.connectionInfo = connectionInfo; this.relationKey = relationKey; this.overwriteTable = overwriteTable; - this.partitionFunction = partitionFunction; + this.distributeFunction = distributeFunction; /* Sanity check arguments -- cannot create an index in append mode. */ Preconditions.checkArgument( overwriteTable || indexes == null || indexes.size() == 0, "Cannot create indexes when appending to a relation."); - /* - * 1) construct immutable copies of the given indexes. - * - * 2) ensure that the index requests are valid: - * - * - lists of column references must be non-null. - * - * - column references are unique per index. - */ + /* 1) construct immutable copies of the given indexes. 2) ensure that the index requests are valid: - lists of + * column references must be non-null. - column references are unique per index. */ if (indexes != null) { ImmutableList.Builder> index = ImmutableList.builder(); for (List i : indexes) { @@ -177,9 +164,7 @@ protected void childEOS() throws DbException { @Override protected void childEOI() throws DbException {} - /** - * @return the name of the relation that this operator will write to. - */ + /** @return the name of the relation that this operator will write to. */ public RelationKey getRelationKey() { return relationKey; } @@ -189,7 +174,7 @@ public Map writeSet() { return ImmutableMap.of( relationKey, new RelationWriteMetadata( - relationKey, getSchema(), overwriteTable, true, partitionFunction)); + relationKey, getSchema(), overwriteTable, true, distributeFunction)); } /** Updates connection information with the environment variables. */ diff --git a/src/edu/washington/escience/myria/operator/network/CollectConsumer.java b/src/edu/washington/escience/myria/operator/network/CollectConsumer.java deleted file mode 100644 index 0e032bd49..000000000 --- a/src/edu/washington/escience/myria/operator/network/CollectConsumer.java +++ /dev/null @@ -1,52 +0,0 @@ -package edu.washington.escience.myria.operator.network; - -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import edu.washington.escience.myria.Schema; -import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.util.MyriaArrayUtils; - -/** - * The consumer part of the Collect Exchange operator. - * - * A Collect operator collects tuples from all the workers. There is a collect producer on each worker, and a collect - * consumer on the server and a master worker if a master worker is needed. - * - * The consumer passively collects Tuples from all the paired CollectProducers - * - */ -public final class CollectConsumer extends Consumer { - - /** The logger for this class. */ - private static final Logger LOGGER = LoggerFactory.getLogger(CollectConsumer.class); - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * @param schema input/output data schema - * @param operatorID my operatorID - * @param workerIDs from which workers the data will come. - * */ - public CollectConsumer( - final Schema schema, final ExchangePairID operatorID, final Set workerIDs) { - super(schema, operatorID, workerIDs); - LOGGER.trace("created CollectConsumer for ExchangePairId=" + operatorID); - } - - /** - * @param schema input/output data schema - * @param operatorID my operatorID - * @param workerIDs from which workers the data will come. - * */ - public CollectConsumer( - final Schema schema, final ExchangePairID operatorID, final int[] workerIDs) { - this( - schema, - operatorID, - MyriaArrayUtils.checkSet(org.apache.commons.lang3.ArrayUtils.toObject(workerIDs))); - } -} diff --git a/src/edu/washington/escience/myria/operator/network/CollectProducer.java b/src/edu/washington/escience/myria/operator/network/CollectProducer.java index 4c9a2aa32..71835eee1 100644 --- a/src/edu/washington/escience/myria/operator/network/CollectProducer.java +++ b/src/edu/washington/escience/myria/operator/network/CollectProducer.java @@ -1,42 +1,25 @@ package edu.washington.escience.myria.operator.network; import edu.washington.escience.myria.operator.Operator; -import edu.washington.escience.myria.operator.network.partition.FixValuePartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.storage.TupleBatch; -/** - * The producer part of the Collect Exchange operator. - * - * The producer actively pushes the tuples generated by the child operator to the paired CollectConsumer. - * - */ +/** The producer part of the Collect Exchange operator. The producer actively pushes the tuples generated by the child + * operator to the paired CollectConsumer. */ public class CollectProducer extends GenericShuffleProducer { - /** A collector always has one partition, no replication, and exactly one outgoing channel of index 0. */ - private static final int[][] COLLECTOR_PARTITION_TO_CHANNEL = new int[][] {{0}}; - /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** - * @param child the child who provides data for this producer to distribute. + /** @param child the child who provides data for this producer to distribute. * @param operatorID destination operator the data goes - * @param collectConsumerWorkerID destination worker the data goes. - * */ + * @param collectConsumerWorkerID destination worker the data goes. */ public CollectProducer( final Operator child, final ExchangePairID operatorID, final int collectConsumerWorkerID) { super( child, new ExchangePairID[] {operatorID}, - COLLECTOR_PARTITION_TO_CHANNEL, new int[] {collectConsumerWorkerID}, - new FixValuePartitionFunction(0), - true); - } - - @Override - protected TupleBatch[] getTupleBatchPartitions(final TupleBatch tup) { - return new TupleBatch[] {tup}; + new BroadcastDistributeFunction(1)); } } diff --git a/src/edu/washington/escience/myria/operator/network/Consumer.java b/src/edu/washington/escience/myria/operator/network/Consumer.java index 6a8abd14a..b5a3e5c43 100644 --- a/src/edu/washington/escience/myria/operator/network/Consumer.java +++ b/src/edu/washington/escience/myria/operator/network/Consumer.java @@ -29,10 +29,8 @@ import edu.washington.escience.myria.storage.TupleBatch; import edu.washington.escience.myria.util.MyriaArrayUtils; -/** - * A Consumer is the counterpart of a producer. It collects data from Producers through IPC. A Consumer can have a - * single Producer data source or multiple Producer data sources. - * */ +/** A Consumer is the counterpart of a producer. It collects data from Producers through IPC. A Consumer can have a + * single Producer data source or multiple Producer data sources. */ public class Consumer extends LeafOperator { /** The logger for this class. */ @@ -41,52 +39,31 @@ public class Consumer extends LeafOperator { /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** - * The buffer for receiving input data. - */ + /** The buffer for receiving input data. */ private transient volatile StreamInputBuffer inputBuffer; - /** - * The operatorID of this Consumer. - * */ + /** The operatorID of this Consumer. */ private final ExchangePairID operatorID; - /** - * The output schema. - */ + /** The output schema. */ private Schema schema; - /** - * Recording the worker EOS status. - * */ + /** Recording the worker EOS status. */ private transient BitSet workerEOS; - /** - * Recording the worker EOI status. - * */ + /** Recording the worker EOI status. */ private transient BitSet workerEOI; - /** - * workerID to index. - * */ + /** workerID to index. */ private transient IntIntHashMap workerIdToIndex; - /** - * From which workers to receive data. - * */ + /** From which workers to receive data. */ private final ImmutableSet sourceWorkers; - /** - * if current query execution is in non-blocking mode. - * */ + /** if current query execution is in non-blocking mode. */ private transient boolean nonBlockingExecution; - /** - * The worker this operator is located at. - * - */ + /** The worker this operator is located at. */ private transient LocalFragmentResourceManager taskResourceManager; - /** - * @return my exchange channels. - * @param myWorkerID for parsing self-references. - */ + /** @return my exchange channels. + * @param myWorkerID for parsing self-references. */ public final ImmutableSet getInputChannelIDs(final int myWorkerID) { ImmutableSet.Builder ecB = ImmutableSet.builder(); for (int wID : sourceWorkers) { @@ -99,11 +76,9 @@ public final ImmutableSet getInputChannelIDs(final int myWork return ecB.build(); } - /** - * @param schema output schema. + /** @param schema output schema. * @param operatorID {@link Consumer#operatorID} - * @param workerIDs {@link Consumer#sourceWorkers} - * */ + * @param workerIDs {@link Consumer#sourceWorkers} */ public Consumer(final Schema schema, final ExchangePairID operatorID, final int[] workerIDs) { this( schema, @@ -111,11 +86,9 @@ public Consumer(final Schema schema, final ExchangePairID operatorID, final int[ MyriaArrayUtils.checkSet(org.apache.commons.lang3.ArrayUtils.toObject(workerIDs))); } - /** - * @param schema output schema. + /** @param schema output schema. * @param operatorID {@link Consumer#operatorID} - * @param workerIDs {@link Consumer#sourceWorkers} - * */ + * @param workerIDs {@link Consumer#sourceWorkers} */ public Consumer( final Schema schema, final ExchangePairID operatorID, final Set workerIDs) { this.operatorID = operatorID; @@ -124,14 +97,6 @@ public Consumer( LOGGER.trace("created Consumer for ExchangePairId=" + operatorID); } - /** - * @param schema output schema. - * @param operatorID {@link Consumer#operatorID} - * */ - public Consumer(final Schema schema, final ExchangePairID operatorID) { - this(schema, operatorID, ImmutableSet.of(IPCConnectionPool.SELF_IPC_ID)); - } - @Override public final void cleanup() { taskResourceManager.releaseInputBuffer(this); @@ -161,18 +126,13 @@ protected final void init(final ImmutableMap execEnvVars) throws inputBuffer = taskResourceManager.getInputBuffer(this); } - /** - * - * Retrieve a batch of tuples from the buffer of ExchangeMessages. Wait if the buffer is empty. + /** Retrieve a batch of tuples from the buffer of ExchangeMessages. Wait if the buffer is empty. * * @param blocking if blocking then return only if there's actually a TupleBatch to return or null if EOS. If not - * blocking then return null immediately if there's no data in the input buffer. - * + * blocking then return null immediately if there's no data in the input buffer. * @return Iterator over the new tuples received from the source workers. Return null if all source * workers have sent an end of file message. - * - * @throws InterruptedException a - */ + * @throws InterruptedException a */ final TupleBatch getTuplesNormal(final boolean blocking) throws InterruptedException { int timeToWait = -1; if (!blocking) { @@ -236,17 +196,13 @@ public final void checkEOSAndEOI() { } } - /** - * @return my IPC operatorID. - * */ + /** @return my IPC operatorID. */ public final ExchangePairID getOperatorID() { return operatorID; } - /** - * @param myWorkerID for parsing self-references. - * @return source worker IDs with self-reference parsed. - * */ + /** @param myWorkerID for parsing self-references. + * @return source worker IDs with self-reference parsed. */ public final int[] getSourceWorkers(final int myWorkerID) { int[] result = new int[sourceWorkers.size()]; int idx = 0; @@ -261,20 +217,16 @@ public final int[] getSourceWorkers(final int myWorkerID) { return result; } - /** - * @return my input buffer. - * */ + /** @return my input buffer. */ public final StreamInputBuffer getInputBuffer() { return inputBuffer; } - /** - * Read a single ExchangeMessage from the queue that buffers incoming ExchangeMessages. + /** Read a single ExchangeMessage from the queue that buffers incoming ExchangeMessages. * * @param timeout Wait for at most timeout milliseconds. If the timeout is negative, wait until an element arrives. * @return received data. - * @throws InterruptedException if interrupted. - */ + * @throws InterruptedException if interrupted. */ private IPCMessage.StreamData take(final int timeout) throws InterruptedException { IPCMessage.StreamData result = null; Verify.verifyNotNull(inputBuffer, "inputBuffer should not be null"); @@ -289,9 +241,7 @@ private IPCMessage.StreamData take(final int timeout) throws Interru return result; } - /** - * @return if there's any message buffered. - * */ + /** @return if there's any message buffered. */ public final boolean hasNext() { return !inputBuffer.isEmpty(); } @@ -311,9 +261,7 @@ protected final Schema generateSchema() { return schema; } - /** - * @param schema the schema to set - * */ + /** @param schema the schema to set */ public final void setSchema(final Schema schema) { this.schema = schema; } diff --git a/src/edu/washington/escience/myria/operator/network/EOSController.java b/src/edu/washington/escience/myria/operator/network/EOSController.java index 1815b6a9e..acb8e3c90 100644 --- a/src/edu/washington/escience/myria/operator/network/EOSController.java +++ b/src/edu/washington/escience/myria/operator/network/EOSController.java @@ -65,7 +65,7 @@ public class EOSController extends Producer { * */ public EOSController( final UnionAll child, final ExchangePairID[] idbOpIDs, final int[] workerIDs) { - super(null, idbOpIDs, workerIDs, false); + super(null, idbOpIDs, workerIDs); if (child != null) { setChildren(new Operator[] {child}); } diff --git a/src/edu/washington/escience/myria/operator/network/GenericShuffleConsumer.java b/src/edu/washington/escience/myria/operator/network/GenericShuffleConsumer.java deleted file mode 100644 index 71ba88165..000000000 --- a/src/edu/washington/escience/myria/operator/network/GenericShuffleConsumer.java +++ /dev/null @@ -1,52 +0,0 @@ -package edu.washington.escience.myria.operator.network; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.collect.ImmutableSet; - -import edu.washington.escience.myria.Schema; -import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.util.MyriaArrayUtils; - -/** - * Generic Shuffle Consumer, which supports the encoding of - * - * 1. BroadcastConsumer - * - * 2. ShuffleConsumer - * - * 3.HyperJoinShuffleConsumer - * - */ -public class GenericShuffleConsumer extends Consumer { - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** The logger for this class. */ - static final Logger LOGGER = LoggerFactory.getLogger(GenericShuffleConsumer.class); - - /** - * @param schema input/output data schema - * @param operatorID my operatorID - * @param workerIDs from which workers the data will come. - * */ - public GenericShuffleConsumer( - final Schema schema, final ExchangePairID operatorID, final int[] workerIDs) { - this( - schema, - operatorID, - MyriaArrayUtils.checkSet(org.apache.commons.lang3.ArrayUtils.toObject(workerIDs))); - } - - /** - * @param schema input/output data schema - * @param operatorID my operatorID - * @param workerIDs from which workers the data will come. - * */ - public GenericShuffleConsumer( - final Schema schema, final ExchangePairID operatorID, final ImmutableSet workerIDs) { - super(schema, operatorID, workerIDs); - LOGGER.trace("created GenericShuffleConsumer for ExchangePairId=" + operatorID); - } -} diff --git a/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java b/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java index d6e7274b6..61b3960ad 100644 --- a/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java +++ b/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java @@ -1,14 +1,13 @@ package edu.washington.escience.myria.operator.network; -import com.google.common.base.Preconditions; +import java.util.List; import edu.washington.escience.myria.DbException; import edu.washington.escience.myria.MyriaConstants.ProfilingMode; import edu.washington.escience.myria.operator.Operator; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.storage.TupleBatch; -import edu.washington.escience.myria.util.MyriaArrayUtils; /** * GenericShuffleProducer, which support json encoding of 1. Broadcast Shuffle 2. One to one Shuffle (Shuffle) 3. Hyper @@ -19,124 +18,47 @@ public class GenericShuffleProducer extends Producer { /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** - * the partition function. - * */ - private final PartitionFunction partitionFunction; - - /** - * Partition of cells. - */ - private final int[][] partitionToChannel; - - /** - * Shuffle to the same operator ID on multiple workers. (The old "ShuffleProducer") - * - * @param child the child who provides data for this producer to distribute. - * @param operatorID destination operators the data goes - * @param workerIDs set of destination workers - * @param pf the partition function - */ - public GenericShuffleProducer( - final Operator child, - final ExchangePairID operatorID, - final int[] workerIDs, - final PartitionFunction pf) { - this( - child, - new ExchangePairID[] {operatorID}, - MyriaArrayUtils.create2DVerticalIndex(pf.numPartition()), - workerIDs, - pf, - false); - Preconditions.checkArgument(workerIDs.length == pf.numPartition()); - } - - /** - * First partition data, then for each partition, send it to the set of workers in cellPartition[partition_id] using - * the same operator ID. For BroadcastProducer and HyperShuffleProducer. - * - * @param child the child who provides data for this producer to distribute. - * @param operatorID destination operators the data goes - * @param cellPartition buckets of destination workers the data goes. The set of ids in cellPartition[i] means - * partition i should go to these workers. Since there's only one operator ID, cellPartition is also the - * mapping from partitions to ioChannels. - * @param workerIDs set of destination workers - * @param pf the partition function - * */ - public GenericShuffleProducer( - final Operator child, - final ExchangePairID operatorID, - final int[][] cellPartition, - final int[] workerIDs, - final PartitionFunction pf) { - this(child, new ExchangePairID[] {operatorID}, cellPartition, workerIDs, pf, false); - Preconditions.checkArgument(cellPartition.length == pf.numPartition()); - } + /** the distribute function. */ + private final DistributeFunction distributeFunction; /** * Shuffle to multiple operator IDs on multiple workers. The most generic constructor. * * @param child the child who provides data for this producer to distribute. * @param operatorIDs destination operators the data goes - * @param partitionToChannel partitionToChannel[i] indicates ioChannels that partition i should be written into. * @param workerIDs set of destination workers - * @param pf the partition function - * @param isOneToOneMapping the same as the one in Producer - * */ + * @param df the distribute function + */ public GenericShuffleProducer( final Operator child, final ExchangePairID[] operatorIDs, - final int[][] partitionToChannel, final int[] workerIDs, - final PartitionFunction pf, - final boolean isOneToOneMapping) { - super(child, operatorIDs, workerIDs, isOneToOneMapping); - Preconditions.checkArgument(partitionToChannel.length == pf.numPartition()); - partitionFunction = pf; - setNumOfPartition(pf.numPartition()); - this.partitionToChannel = partitionToChannel; - } - - /** - * @return return partition function. - * */ - public final PartitionFunction getPartitionFunction() { - return partitionFunction; + final DistributeFunction df) { + super(child, operatorIDs, workerIDs); + distributeFunction = df; } @Override protected final void consumeTuples(final TupleBatch tup) throws DbException { - final TupleBatch[] partitions = getTupleBatchPartitions(tup); - + final List> partitions = distributeFunction.distribute(tup); if (getProfilingMode().contains(ProfilingMode.QUERY)) { - for (int partitionIdx = 0; partitionIdx < partitions.length; partitionIdx++) { - if (partitions[partitionIdx] != null) { - final int numTuples = partitions[partitionIdx].numTuples(); - for (int channelId : partitionToChannel[partitionIdx]) { - final int destWorkerId = getOutputIDs()[channelId].getRemoteID(); - getProfilingLogger().recordSent(this, numTuples, destWorkerId); + for (int channelIdx = 0; channelIdx < partitions.size(); channelIdx++) { + int numTuples = 0; + for (TupleBatch tb : partitions.get(channelIdx)) { + if (tb != null) { + numTuples += tb.numTuples(); } } + final int destWorkerId = getOutputIDs()[channelIdx].getRemoteID(); + getProfilingLogger().recordSent(this, numTuples, destWorkerId); } } - writePartitionsIntoChannels(true, partitionToChannel, partitions); - } - - /** - * call partition function to partition this tuple batch as an array of shallow copies of TupleBatch. subclasses can - * override this method to have smarter partition approach. - * - * @param tb the tuple batch to be partitioned. - * @return partitions. - */ - protected TupleBatch[] getTupleBatchPartitions(final TupleBatch tb) { - return tb.partition(partitionFunction); + writePartitionsIntoChannels(partitions); } @Override protected void childEOS() throws DbException { - writePartitionsIntoChannels(false, partitionToChannel, null); + writePartitionsIntoChannels(null); for (int p = 0; p < numChannels(); p++) { super.channelEnds(p); } @@ -144,10 +66,7 @@ protected void childEOS() throws DbException { @Override protected final void childEOI() throws DbException { - TupleBatch[] partitions = new TupleBatch[getNumOfPartition()]; - for (int i = 0; i < getNumOfPartition(); i++) { - partitions[i] = TupleBatch.eoiTupleBatch(getSchema()); - } - writePartitionsIntoChannels(false, partitionToChannel, partitions); + writePartitionsIntoChannels( + distributeFunction.distribute(TupleBatch.eoiTupleBatch(getSchema()))); } } diff --git a/src/edu/washington/escience/myria/operator/network/LocalMultiwayConsumer.java b/src/edu/washington/escience/myria/operator/network/LocalMultiwayConsumer.java deleted file mode 100644 index 86f37cb0b..000000000 --- a/src/edu/washington/escience/myria/operator/network/LocalMultiwayConsumer.java +++ /dev/null @@ -1,36 +0,0 @@ -package edu.washington.escience.myria.operator.network; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import edu.washington.escience.myria.Schema; -import edu.washington.escience.myria.parallel.ExchangePairID; - -/** - * The consumer part of the Collect Exchange operator. - * - * A Collect operator collects tuples from all the workers. There is a collect producer on each worker, and a collect - * consumer on the server and a master worker if a master worker is needed. - * - * The consumer passively collects Tuples from all the paired LocalMultiwayProducers - * - */ -public final class LocalMultiwayConsumer extends Consumer { - - /** The logger for this class. */ - private static final Logger LOGGER = LoggerFactory.getLogger(LocalMultiwayConsumer.class); - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * If there's no child operator, a Schema is needed. - * - * @param schema the input/output data schema - * @param operatorID my operatorID - */ - public LocalMultiwayConsumer(final Schema schema, final ExchangePairID operatorID) { - super(schema, operatorID); - LOGGER.trace("created multiway consumer for ExchangePairID=" + operatorID); - } -} diff --git a/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java b/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java index cfa0df96f..413b10beb 100644 --- a/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java +++ b/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java @@ -1,39 +1,23 @@ package edu.washington.escience.myria.operator.network; import edu.washington.escience.myria.operator.Operator; -import edu.washington.escience.myria.operator.network.partition.FixValuePartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.parallel.ipc.IPCConnectionPool; -import edu.washington.escience.myria.storage.TupleBatch; -import edu.washington.escience.myria.util.MyriaArrayUtils; -/** - * The producer part of the Collect Exchange operator. - * - * The producer actively pushes the tuples generated by the child operator to the paired LocalMultiwayConsumer. - * - */ +/** A producer that duplicates tuple batches to corresponding LocalMultiwayConsumers. */ public final class LocalMultiwayProducer extends GenericShuffleProducer { /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** - * @param child the child who provides data for this producer to distribute. - * @param operatorIDs destination operators the data goes - * */ + /** @param child the child who provides data for this producer to distribute. + * @param operatorIDs destination operators the data goes */ public LocalMultiwayProducer(final Operator child, final ExchangePairID[] operatorIDs) { super( child, operatorIDs, - MyriaArrayUtils.create2DHorizontalIndex(operatorIDs.length), new int[] {IPCConnectionPool.SELF_IPC_ID}, - new FixValuePartitionFunction(0), - false); - } - - @Override - protected TupleBatch[] getTupleBatchPartitions(final TupleBatch tup) { - return new TupleBatch[] {tup}; + new BroadcastDistributeFunction(operatorIDs.length)); } } diff --git a/src/edu/washington/escience/myria/operator/network/LocalShuffleConsumer.java b/src/edu/washington/escience/myria/operator/network/LocalShuffleConsumer.java deleted file mode 100644 index af059c518..000000000 --- a/src/edu/washington/escience/myria/operator/network/LocalShuffleConsumer.java +++ /dev/null @@ -1,26 +0,0 @@ -package edu.washington.escience.myria.operator.network; - -import edu.washington.escience.myria.Schema; -import edu.washington.escience.myria.parallel.ExchangePairID; - -/** - * The consumer part of the Shuffle Exchange operator. - * - * A ShuffleProducer operator sends tuples to all the workers according to some PartitionFunction, while the - * ShuffleConsumer (this class) encapsulates the methods to collect the tuples received at the worker from multiple - * source workers' ShuffleProducer. - * - */ -public final class LocalShuffleConsumer extends Consumer { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * @param schema input/output data schema - * @param operatorID source operatorID - * */ - public LocalShuffleConsumer(final Schema schema, final ExchangePairID operatorID) { - super(schema, operatorID); - } -} diff --git a/src/edu/washington/escience/myria/operator/network/LocalShuffleProducer.java b/src/edu/washington/escience/myria/operator/network/LocalShuffleProducer.java deleted file mode 100644 index a50470ad5..000000000 --- a/src/edu/washington/escience/myria/operator/network/LocalShuffleProducer.java +++ /dev/null @@ -1,39 +0,0 @@ -package edu.washington.escience.myria.operator.network; - -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.operator.Operator; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; -import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.parallel.ipc.IPCConnectionPool; -import edu.washington.escience.myria.util.MyriaArrayUtils; - -/** - * The producer part of the Shuffle Exchange operator. - * - * ShuffleProducer distributes tuples to the workers according to some partition function (provided as a - * PartitionFunction object during the ShuffleProducer's instantiation). - * - */ -public class LocalShuffleProducer extends GenericShuffleProducer { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * @param child the child who provides data for this producer to distribute. - * @param operatorIDs destination operators the data goes - * @param pf the partition function - * */ - public LocalShuffleProducer( - final Operator child, final ExchangePairID[] operatorIDs, final PartitionFunction pf) { - super( - child, - operatorIDs, - MyriaArrayUtils.create2DVerticalIndex(pf.numPartition()), - new int[] {IPCConnectionPool.SELF_IPC_ID}, - pf, - false); - Preconditions.checkArgument(operatorIDs.length == pf.numPartition()); - } -} diff --git a/src/edu/washington/escience/myria/operator/network/Producer.java b/src/edu/washington/escience/myria/operator/network/Producer.java index 46de256aa..f136bcfcb 100644 --- a/src/edu/washington/escience/myria/operator/network/Producer.java +++ b/src/edu/washington/escience/myria/operator/network/Producer.java @@ -6,7 +6,6 @@ import org.jboss.netty.channel.ChannelFuture; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import edu.washington.escience.myria.DbException; @@ -29,7 +28,6 @@ import edu.washington.escience.myria.parallel.ipc.StreamOutputChannel; import edu.washington.escience.myria.storage.TupleBatch; import edu.washington.escience.myria.storage.TupleBatchBuffer; -import edu.washington.escience.myria.util.MyriaArrayUtils; /** * A Producer is the counterpart of a consumer. It dispatch data using IPC channels to Consumers. Like network socket, @@ -40,23 +38,15 @@ public abstract class Producer extends RootOperator implements StreamingStateful /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** - * The worker this operator is located at. - */ + /** The worker this operator is located at. */ private transient LocalFragmentResourceManager taskResourceManager; - /** - * the netty channels doing the true IPC IO. - */ + /** the netty channels doing the true IPC IO. */ private transient StreamOutputChannel[] ioChannels; - /** - * if the corresponding ioChannel is available to write again. - */ + /** if the corresponding ioChannel is available to write again. */ private transient boolean[] ioChannelsAvail; - /** - * output buffers of partitions. - */ + /** output buffers of partitions. */ private transient TupleBatchBuffer[] partitionBuffers; /** tried to send tuples for each channel. */ @@ -64,19 +54,13 @@ public abstract class Producer extends RootOperator implements StreamingStateful /** pending tuples to be sent for each channel. */ private transient List> pendingTuplesToSend; - /** - * output channel IDs. - */ + /** output channel IDs. */ private final StreamIOChannelID[] outputIDs; - /** - * localized output stream channel IDs, with self references dereferenced. - */ + /** localized output stream channel IDs, with self references dereferenced. */ private transient StreamIOChannelID[] localizedOutputIDs; - /** - * if current query execution is in non-blocking mode. - */ + /** if current query execution is in non-blocking mode. */ private transient boolean nonBlockingExecution; /** number of parition, by default 1. */ @@ -85,67 +69,6 @@ public abstract class Producer extends RootOperator implements StreamingStateful /** if the outgoing channels are totally local. */ private boolean totallyLocal; - /** - * no worker means to the owner worker. - * - * @param child the child providing data. - * @param oIDs operator IDs. - */ - public Producer(final Operator child, final ExchangePairID[] oIDs) { - this( - child, - oIDs, - MyriaArrayUtils.arrayFillAndReturn(new int[oIDs.length], IPCConnectionPool.SELF_IPC_ID), - true); - } - - /** - * the same oID to different workers (shuffle or copy). - * - * @param oID the operator ID. - * @param child the child providing data. - * @param destinationWorkerIDs worker IDs. - * - */ - public Producer( - final Operator child, final ExchangePairID oID, final int[] destinationWorkerIDs) { - this( - child, - (ExchangePairID[]) - MyriaArrayUtils.arrayFillAndReturn( - new ExchangePairID[destinationWorkerIDs.length], oID), - destinationWorkerIDs, - true); - } - - /** - * same worker with different oIDs (multiway copy). - * - * @param oIDs the operator IDs. - * @param child the child providing data. - * @param destinationWorkerID the worker ID. - */ - public Producer( - final Operator child, final ExchangePairID[] oIDs, final int destinationWorkerID) { - this( - child, - oIDs, - MyriaArrayUtils.arrayFillAndReturn( - new int[oIDs.length], Integer.valueOf(destinationWorkerID)), - true); - } - - /** - * A single oID to a single worker (collect). - * - * @param oID the operator ID. - * @param child the child providing data. - * @param destinationWorkerID the worker ID. - */ - public Producer(final Operator child, final ExchangePairID oID, final int destinationWorkerID) { - this(child, new ExchangePairID[] {oID}, new int[] {destinationWorkerID}, true); - } - /** * Two modes: *

@@ -160,34 +83,20 @@ public Producer(final Operator child, final ExchangePairID oID, final int destin * The number of producer channels is oID.length*destinationWorkerIDs.length * * - * * @param oIDs the operator IDs. * @param child the child providing data. * @param destinationWorkerIDs the worker IDs. * @param isOne2OneMapping choosing the mode. - * */ public Producer( - final Operator child, - final ExchangePairID[] oIDs, - final int[] destinationWorkerIDs, - final boolean isOne2OneMapping) { + final Operator child, final ExchangePairID[] oIDs, final int[] destinationWorkerIDs) { super(child); - if (isOne2OneMapping) { - // oID and worker pairs. each ( oIDs[i], destinationWorkerIDs[i] ) pair is a logical channel. - Preconditions.checkArgument(oIDs.length == destinationWorkerIDs.length); - outputIDs = new StreamIOChannelID[oIDs.length]; - for (int i = 0; i < oIDs.length; i++) { - outputIDs[i] = new StreamIOChannelID(oIDs[i].getLong(), destinationWorkerIDs[i]); - } - } else { - outputIDs = new StreamIOChannelID[oIDs.length * destinationWorkerIDs.length]; - int idx = 0; - for (int wID : destinationWorkerIDs) { - for (ExchangePairID oID : oIDs) { - outputIDs[idx] = new StreamIOChannelID(oID.getLong(), wID); - idx++; - } + outputIDs = new StreamIOChannelID[oIDs.length * destinationWorkerIDs.length]; + int idx = 0; + for (int wID : destinationWorkerIDs) { + for (ExchangePairID oID : oIDs) { + outputIDs[idx] = new StreamIOChannelID(oID.getLong(), wID); + idx++; } } totallyLocal = true; @@ -201,9 +110,7 @@ public Producer( setBackupBuffer(new SimpleAppenderStateEncoding()); } - /** - * @return the outputIDs - */ + /** @return the outputIDs */ protected StreamIOChannelID[] getOutputIDs() { return outputIDs; } @@ -295,7 +202,6 @@ public void setStreamingStates(final List states) { * set backup buffers. * * @param stateEncoding the streaming state encoding. - * */ public void setBackupBuffer(final StreamingStateEncoding stateEncoding) { List states = new ArrayList(); @@ -343,73 +249,23 @@ protected final ChannelFuture writeMessage(final int chIdx, final TupleBatch msg } /** - * Pop tuple batches from each of the buffers and try to write them to corresponding channels, if possible. - * - * @param usingTimeout use popAny() or popAnyUsingTimeout() when poping - * @param partitions the list of partitions as tuple batches. - */ - protected final void writePartitionsIntoChannels( - final boolean usingTimeout, final TupleBatch[] partitions) { - writePartitionsIntoChannels( - usingTimeout, MyriaArrayUtils.create2DVerticalIndex(numChannels()), partitions); - } - - /** - * Pop tuple batches from each of the buffers and try to write them to corresponding channels, if possible. + * Pop tuple batches from each of the buffers and try to write them to corresponding channels if possible. * - * @param usingTimeout use {@link TupleBatchBuffer#popAny()} or {@link TupleBatchBuffer#popAnyUsingTimeout()} when - * popping - * @param channelIndices the same as {@link GenericShuffleProducer#cellPartition}. * @param partitions the list of partitions as tuple batches. */ - protected final void writePartitionsIntoChannels( - final boolean usingTimeout, final int[][] channelIndices, final TupleBatch[] partitions) { + protected final void writePartitionsIntoChannels(final List> partitions) { FTMode mode = taskResourceManager.getFragment().getLocalSubQuery().getFTMode(); - - if (totallyLocal) { - if (partitions != null) { - for (int i = 0; i < numOfPartition; ++i) { - if (partitions[i] != null) { - for (int j : channelIndices[i]) { - if (!ioChannelsAvail[j] && mode.equals(FTMode.ABANDON)) { - continue; - } - pendingTuplesToSend.get(j).add(partitions[i]); - } - } - } - } - } else { - if (partitions != null) { - for (int i = 0; i < numOfPartition; ++i) { - if (partitions[i] != null) { - partitionBuffers[i].absorb(partitions[i]); - } - } + for (int i = 0; i < numChannels(); ++i) { + if (!ioChannelsAvail[i] && mode.equals(FTMode.ABANDON)) { + continue; } - for (int i = 0; i < numOfPartition; ++i) { - while (true) { - TupleBatch tb = null; - if (usingTimeout) { - tb = partitionBuffers[i].popAnyUsingTimeout(); - } else { - tb = partitionBuffers[i].popAny(); - } - if (tb == null) { - break; - } - for (int j : channelIndices[i]) { - if (!ioChannelsAvail[j] && mode.equals(FTMode.ABANDON)) { - continue; - } - pendingTuplesToSend.get(j).add(tb); + if (partitions != null) + for (TupleBatch tb : partitions.get(i)) { + if (tb != null) { + pendingTuplesToSend.get(i).add(tb); } } - } - } - - for (int i = 0; i < numChannels(); ++i) { - if (!ioChannelsAvail[i] && (mode.equals(FTMode.ABANDON) || mode.equals(FTMode.REJOIN))) { + if (!ioChannelsAvail[i] && mode.equals(FTMode.REJOIN)) { continue; } while (true) { @@ -418,7 +274,7 @@ protected final void writePartitionsIntoChannels( break; } if (mode.equals(FTMode.REJOIN) && !(this instanceof LocalMultiwayProducer)) { - // rejoin, append the TB into the backup buffer in case of recovering + // append the TB into the backup buffer for recovery tb = triedToSendTuples.get(i).update(tb); } try { @@ -426,10 +282,7 @@ protected final void writePartitionsIntoChannels( writeMessage(i, tb); } } catch (IllegalStateException e) { - if (mode.equals(FTMode.ABANDON)) { - ioChannelsAvail[i] = false; - break; - } else if (mode.equals(FTMode.REJOIN)) { + if (mode.equals(FTMode.ABANDON) || mode.equals(FTMode.REJOIN)) { ioChannelsAvail[i] = false; break; } else { @@ -440,9 +293,7 @@ protected final void writePartitionsIntoChannels( } } - /** - * @return the number of tuples in all buffers. - */ + /** @return the number of tuples in all buffers. */ public final long getNumTuplesInBuffers() { long sum = 0; for (StreamingState state : triedToSendTuples) { @@ -494,16 +345,12 @@ public final StreamIOChannelID[] getOutputChannelIDs(final int myWorkerID) { return result; } - /** - * @return number of output channels. - */ + /** @return number of output channels. */ public final int numChannels() { return ioChannels.length; } - /** - * @return The resource manager of the running task. - */ + /** @return The resource manager of the running task. */ protected LocalFragmentResourceManager getTaskResourceManager() { return taskResourceManager; } @@ -546,23 +393,17 @@ public final List getChannelIndicesOfAWorker(final int workerId) { return ret; } - /** - * @return the channel availability array. - */ + /** @return the channel availability array. */ public boolean[] getChannelsAvail() { return ioChannelsAvail; } - /** - * @return the channel array. - */ + /** @return the channel array. */ public StreamOutputChannel[] getChannels() { return ioChannels; } - /** - * process EOS and EOI logic. - */ + /** process EOS and EOI logic. */ @Override protected final void checkEOSAndEOI() { Operator child = getChild(); @@ -573,7 +414,8 @@ protected final void checkEOSAndEOI() { if (taskResourceManager.getFragment().getLocalSubQuery().getFTMode().equals(FTMode.REJOIN)) { for (LinkedList tbs : pendingTuplesToSend) { if (tbs.size() > 0) { - // due to failure, buffers are not empty, this task needs to be executed again to push these TBs out when + // due to failure, buffers are not empty, this task + // needs to be executed again to push these TBs out when // channels are available return; } @@ -593,17 +435,12 @@ public void setNumOfPartition(final int num) { numOfPartition = num; } - /** - * - * @return the number of partitions. - */ + /** @return the number of partitions. */ public int getNumOfPartition() { return numOfPartition; } - /** - * @return the number of tuples written to channels. - */ + /** @return the number of tuples written to channels. */ public final long getNumTuplesWrittenToChannels() { return numTuplesWrittenToChannels; } diff --git a/src/edu/washington/escience/myria/operator/network/RecoverProducer.java b/src/edu/washington/escience/myria/operator/network/RecoverProducer.java index 30466a3bf..5a72f6f6f 100644 --- a/src/edu/washington/escience/myria/operator/network/RecoverProducer.java +++ b/src/edu/washington/escience/myria/operator/network/RecoverProducer.java @@ -3,18 +3,14 @@ import com.google.common.base.Preconditions; import edu.washington.escience.myria.DbException; -import edu.washington.escience.myria.operator.Operator; import edu.washington.escience.myria.operator.BatchTupleSource; +import edu.washington.escience.myria.operator.Operator; import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.parallel.ipc.StreamOutputChannel; import edu.washington.escience.myria.storage.TupleBatch; -/** - * The producer part of the Collect Exchange operator. - * - * The producer actively pushes the tuples generated by the child operator to the paired CollectConsumer. - * - */ +/** The producer part of the Collect Exchange operator. The producer actively pushes the tuples generated by the child + * operator to the paired CollectConsumer. */ public final class RecoverProducer extends CollectProducer { /** Required for Java serialization. */ @@ -29,13 +25,11 @@ public final class RecoverProducer extends CollectProducer { /** the channel index that this operator is recovering for. */ private final int channelIndx; - /** - * @param child the child who provides data for this producer to distribute. + /** @param child the child who provides data for this producer to distribute. * @param operatorID destination operator the data goes * @param collectConsumerWorkerID destination worker the data goes. * @param oriProducer the original producer. - * @param channelIndx the channel index that this operator is recovering for. * - * */ + * @param channelIndx the channel index that this operator is recovering for. * */ public RecoverProducer( final Operator child, final ExchangePairID operatorID, @@ -49,7 +43,7 @@ public RecoverProducer( @Override protected void childEOS() throws DbException { - writePartitionsIntoChannels(false, null); + writePartitionsIntoChannels(null); Preconditions.checkArgument(getChild() instanceof BatchTupleSource); if (!oriProducer.eos()) { StreamOutputChannel tmp = getChannels()[0]; diff --git a/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java new file mode 100644 index 000000000..700250119 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java @@ -0,0 +1,36 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import edu.washington.escience.myria.util.MyriaArrayUtils; + +/** + * Broadcast distribute function. + */ +public final class BroadcastDistributeFunction extends DistributeFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** + */ + @JsonCreator + public BroadcastDistributeFunction() { + super(new SinglePartitionFunction(), null); + } + + /** + * @param numDestinations number of destination + */ + public BroadcastDistributeFunction(final int numDestinations) { + this(); + partitionToDestination = MyriaArrayUtils.create2DHorizontalIndexList(numDestinations); + this.numDestinations = numDestinations; + } + + @Override + public void setNumDestinations(final int numWorker, final int numOperatorId) { + partitionToDestination = MyriaArrayUtils.create2DHorizontalIndexList(numWorker * numOperatorId); + numDestinations = numWorker * numOperatorId; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java new file mode 100644 index 000000000..c9dec02a0 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java @@ -0,0 +1,82 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import javax.annotation.Nonnull; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.Lists; + +import edu.washington.escience.myria.storage.TupleBatch; + +/** + * A dataset is distributed by two steps: First, using a partition function to generate a partition for each tuple; + * Second, mapping each partition to a set of destinations. A destination corresponds to an output channel, which is + * determined by a worker ID + an operator ID. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonSubTypes({ + @Type(value = BroadcastDistributeFunction.class, name = "Broadcast"), + @Type(value = HyperCubeDistributeFunction.class, name = "HyperCube"), + @Type(value = HashDistributeFunction.class, name = "Hash"), + @Type(value = RoundRobinDistributeFunction.class, name = "RoundRobin"), + @Type(value = IdentityDistributeFunction.class, name = "Identity") +}) +public abstract class DistributeFunction implements Serializable { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** The partition function. */ + protected PartitionFunction partitionFunction; + + /** The mapping from partitions to destinations. */ + protected List> partitionToDestination; + + /** number of destinations. */ + protected int numDestinations; + + /** + * @param partitionFunction partition function. + * @param partitionToDestination mapping from partitions to destinations + */ + public DistributeFunction( + final PartitionFunction partitionFunction, final List> partitionToDestination) { + this.partitionFunction = partitionFunction; + this.partitionToDestination = partitionToDestination; + } + + /** + * @param data the input data + * @return a list of tuple batch lists, each represents output data of one destination. + */ + public List> distribute(@Nonnull final TupleBatch data) { + List> result = new ArrayList>(); + if (data.isEOI()) { + for (int i = 0; i < numDestinations; ++i) { + result.add(Lists.newArrayList(data)); + } + } else { + for (int i = 0; i < numDestinations; ++i) { + result.add(new ArrayList()); + } + TupleBatch[] tbs = partitionFunction.partition(data); + for (int i = 0; i < tbs.length; ++i) { + for (int channelIdx : partitionToDestination.get(i)) { + result.get(channelIdx).add(tbs[i]); + } + } + } + return result; + } + + /** + * @param numWorker the number of workers to distribute on + * @param numOperatorId the number of involved operator IDs + */ + public abstract void setNumDestinations(final int numWorker, final int numOperatorId); +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java new file mode 100644 index 000000000..ace1078db --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java @@ -0,0 +1,40 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import edu.washington.escience.myria.util.MyriaArrayUtils; + +/** + * Hash distribute function. + */ +public final class HashDistributeFunction extends DistributeFunction { + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** The indices used for partitioning. */ + @JsonProperty private final int[] indexes; + + /** + * @param indexes + */ + @JsonCreator + public HashDistributeFunction(@JsonProperty("indexes") final int[] indexes) { + super(new HashPartitionFunction(indexes), null); + this.indexes = indexes; + } + + @Override + public void setNumDestinations(final int numWorker, final int numOperatorId) { + partitionFunction.setNumPartitions(numWorker); + partitionToDestination = MyriaArrayUtils.create2DVerticalIndexList(numWorker); + numDestinations = numWorker; + } + + /** + * @return indexes + */ + public int[] getIndexes() { + return indexes; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java new file mode 100644 index 000000000..db6ad2a66 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java @@ -0,0 +1,79 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.util.BitSet; + +import javax.annotation.Nonnull; + +import com.google.common.base.Preconditions; + +import edu.washington.escience.myria.storage.TupleBatch; +import edu.washington.escience.myria.util.HashUtils; + +/** + * Implementation that uses multiple fields as the key to hash The partition of a tuple is decided by the hash code of a + * group of fields of the tuple. + */ +public final class HashPartitionFunction extends PartitionFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** The indices used for partitioning. */ + // @JsonProperty + private final int[] indexes; + + /** The index of the chosen hashcode in HashUtils. */ + // @JsonProperty + private final int seedIndex; + + /** + * @param indexes the indices used for partitioning. + */ + // @JsonCreator + // public HashPartitionFunction(@JsonProperty("indexes") final int[] indexes) { + public HashPartitionFunction(final int[] indexes) { + this(indexes, 0); + } + + /** + * @param indexes the indices used for partitioning. + * @param seedIndex the index of chosen hash seed. + */ + public HashPartitionFunction(final int[] indexes, final int seedIndex) { + Preconditions.checkArgument( + indexes.length > 0, "HashPartitionFunction requires at least 1 field to hash"); + for (int i = 0; i < indexes.length; ++i) { + Preconditions.checkArgument( + indexes[i] >= 0, + "HashPartitionFunction field index %s cannot take negative value %s", + i, + indexes[i]); + } + this.indexes = indexes; + this.seedIndex = seedIndex % HashUtils.NUM_OF_HASHFUNCTIONS; + } + + /** + * @return the field indexes on which tuples will be hash partitioned. + */ + public int[] getIndexes() { + return indexes; + } + + @Override + public TupleBatch[] partition(@Nonnull final TupleBatch tb) { + BitSet[] partitions = new BitSet[numPartition()]; + for (int i = 0; i < partitions.length; ++i) { + partitions[i] = new BitSet(); + } + for (int i = 0; i < tb.numTuples(); i++) { + int p = Math.floorMod(HashUtils.hashSubRow(tb, indexes, i, seedIndex), numPartition()); + partitions[p].set(i); + } + TupleBatch[] tbs = new TupleBatch[numPartition()]; + for (int i = 0; i < tbs.length; ++i) { + tbs[i] = tb.filter(partitions[i]); + } + return tbs; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java b/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java new file mode 100644 index 000000000..0d76dcb4c --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java @@ -0,0 +1,50 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.util.List; + +import javax.annotation.Nullable; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Ints; + +/** How is a dataset distributed. */ +public class HowDistributed { + + /** The distribute function used to distirbute the dataset. Null means unknown. */ + @JsonProperty private DistributeFunction df = null; + /** The sequence of workers that the dataset is partitioned on. Null means unknown. */ + @JsonProperty private ImmutableList workers = null; + + /** @param df the distribute function. + * @param workers the sequence of workers. */ + public HowDistributed(@Nullable final DistributeFunction df, @Nullable final int[] workers) { + this.df = df; + if (workers != null) { + this.workers = ImmutableList.copyOf(Ints.asList(workers)); + } + } + + /** Static function to create a HowPartitioned object. + * + * @param df the distribute function. + * @param workers the sequence of workers. * + * @return a new HowPartitioned reference to the specified relation. */ + @JsonCreator + public static HowDistributed of( + @JsonProperty("df") final DistributeFunction df, + @JsonProperty("workers") final int[] workers) { + return new HowDistributed(df, workers); + } + + /** @return the distribute function. */ + public DistributeFunction getDf() { + return df; + } + + /** @return the workers. */ + public List getWorkers() { + return workers; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java new file mode 100644 index 000000000..b81ece120 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java @@ -0,0 +1,45 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.util.ArrayList; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** Distribute function for HyperCube Shuffle. */ +public final class HyperCubeDistributeFunction extends DistributeFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** + * @param hyperCubeDimensions the sizes of each dimension of the hypercube. + * @param hashedColumns which fields are hashed. + * @param mappedHCDimensions mapped hypercube dimensions of hashed columns. + * @param cellPartition mapping from cells to destinations. + */ + @JsonCreator + public HyperCubeDistributeFunction( + @JsonProperty("hyperCubeDimensions") final int[] hyperCubeDimensions, + @JsonProperty("hashedColumns") final int[] hashedColumns, + @JsonProperty("mappedHCDimensions") final int[] mappedHCDimensions, + @JsonProperty("callPartition") final List> cellPartition) { + super( + new HyperCubePartitionFunction(hyperCubeDimensions, hashedColumns, mappedHCDimensions), + cellPartition); + } + + /** @return all destinations */ + public List getAllDestinations() { + List values = new ArrayList(); + for (List l : partitionToDestination) { + values.addAll(l); + } + return values; + } + + @Override + public void setNumDestinations(final int numWorker, final int numOperatorId) { + numDestinations = getAllDestinations().size(); + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java new file mode 100644 index 000000000..4c14add19 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java @@ -0,0 +1,57 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.util.BitSet; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import edu.washington.escience.myria.storage.TupleBatch; +import edu.washington.escience.myria.util.HashUtils; + +/** + * Multiple field multiple dimension hash partition function for HyperCubeJoin. + */ +public final class HyperCubePartitionFunction extends PartitionFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + @JsonProperty private final int[] hashedColumns; + @JsonProperty private final int[] mappedHCDimensions; + @JsonProperty private final int[] hyperCubeDimensions; + + /** + * @param hyperCubeDimensions the sizes of each dimension of the hypercube. + * @param hashedColumns which fields are hashed. + * @param mappedHCDimensions mapped hypercube dimensions of hashed columns. + */ + public HyperCubePartitionFunction( + final int[] hyperCubeDimensions, final int[] hashedColumns, final int[] mappedHCDimensions) { + super(); + this.hashedColumns = hashedColumns; + this.hyperCubeDimensions = hyperCubeDimensions; + this.mappedHCDimensions = mappedHCDimensions; + } + + @Override + public TupleBatch[] partition(final TupleBatch tb) { + BitSet[] partitions = new BitSet[numPartition()]; + for (int i = 0; i < tb.numTuples(); i++) { + int p = 0; + for (int j = 0; j < hashedColumns.length; j++) { + p += + Math.floorMod( + HashUtils.hashSubRow(tb, new int[] {hashedColumns[j]}, i, mappedHCDimensions[j]), + hyperCubeDimensions[mappedHCDimensions[j]]); + if (p != hashedColumns.length - 1) { + p *= hyperCubeDimensions[mappedHCDimensions[j]]; + } + } + partitions[p].set(i); + } + TupleBatch[] tbs = new TupleBatch[numPartition()]; + for (int i = 0; i < tbs.length; ++i) { + tbs[i] = tb.filter(partitions[i]); + } + return tbs; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java new file mode 100644 index 000000000..c1e5cf3c0 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java @@ -0,0 +1,31 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import edu.washington.escience.myria.util.MyriaArrayUtils; + +/** + * Implementation of a DistributeFunction that use the trivial identity hash. (i.e. a --> a) The attribute to hash on + * must be an INT column and should represent a workerID + */ +public final class IdentityDistributeFunction extends DistributeFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** + * @param index the column index for distributing + */ + @JsonCreator + public IdentityDistributeFunction(@JsonProperty("index") final int index) { + super(new IdentityPartitionFunction(index), null); + } + + @Override + public void setNumDestinations(final int numWorker, final int numOperatorId) { + partitionFunction.setNumPartitions(numWorker); + partitionToDestination = MyriaArrayUtils.create2DVerticalIndexList(numWorker); + numDestinations = numWorker; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java new file mode 100644 index 000000000..5df0141f6 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java @@ -0,0 +1,42 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.util.BitSet; + +import javax.annotation.Nonnull; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import edu.washington.escience.myria.storage.TupleBatch; + +/** + * Implementation that uses multiple fields as the key to hash The partition of + * a tuple is decided by the hash code of a group of fields of the tuple. + */ +public final class IdentityPartitionFunction extends PartitionFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** the column index for partitioning. */ + @JsonProperty private final int index; + + /** + * @param index the column index for partitioning. + */ + public IdentityPartitionFunction(final int index) { + this.index = index; + } + + @Override + public TupleBatch[] partition(@Nonnull final TupleBatch tb) { + BitSet[] partitions = new BitSet[numPartition()]; + for (int i = 0; i < tb.numTuples(); i++) { + partitions[tb.getInt(index, i) - 1].set(i); + } + TupleBatch[] tbs = new TupleBatch[numPartition()]; + for (int i = 0; i < tbs.length; ++i) { + tbs[i] = tb.filter(partitions[i]); + } + return tbs; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java new file mode 100644 index 000000000..c7fae59c1 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java @@ -0,0 +1,50 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.io.Serializable; + +import javax.annotation.Nonnull; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Preconditions; + +import edu.washington.escience.myria.storage.TupleBatch; + +/** The ShuffleProducer class uses an instance of the PartitionFunction class to decide which worker a tuple should be + * routed to. Typically, the ShuffleProducer class invokes {@link #partition(Tuple, Schema) partition} on every tuple it + * generates. */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonSubTypes({ + @Type(value = RoundRobinPartitionFunction.class, name = "RoundRobin"), + @Type(value = IdentityPartitionFunction.class, name = "Identity"), + @Type(value = HyperCubePartitionFunction.class, name = "HyperCube"), + @Type(value = HashPartitionFunction.class, name = "Hash"), + @Type(value = SinglePartitionFunction.class, name = "Single") +}) +public abstract class PartitionFunction implements Serializable { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** number of partitions. */ + private Integer numPartitions = null; + + /** @return the number of partitions. */ + public final int numPartition() { + Preconditions.checkState(numPartitions != null, "numPartitions has not been set"); + return numPartitions; + } + + /** partition the tuple batch into TupleBatch[], each element is one partition. + * + * @param data the data to be partitioned. + * @return an array of partitions. */ + public abstract TupleBatch[] partition(@Nonnull final TupleBatch data); + + /** @param numPartitions the number of partitions. */ + public final void setNumPartitions(final int numPartitions) { + Preconditions.checkArgument(numPartitions > 0, "numPartitions must be > 0"); + this.numPartitions = numPartitions; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java new file mode 100644 index 000000000..7b54b3537 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java @@ -0,0 +1,28 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import edu.washington.escience.myria.util.MyriaArrayUtils; + +/** + * Round robin distribute function. + */ +public final class RoundRobinDistributeFunction extends DistributeFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** + */ + @JsonCreator + public RoundRobinDistributeFunction() { + super(new RoundRobinPartitionFunction(), null); + } + + @Override + public void setNumDestinations(final int numWorker, final int numOperatorId) { + partitionFunction.setNumPartitions(numWorker); + partitionToDestination = MyriaArrayUtils.create2DVerticalIndexList(numWorker); + numDestinations = numWorker; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java new file mode 100644 index 000000000..5ca1ca115 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java @@ -0,0 +1,32 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import java.util.BitSet; + +import javax.annotation.Nonnull; + +import edu.washington.escience.myria.storage.TupleBatch; + +/** + * A partition function that simply sends one tuple to each output in turn. + */ +public final class RoundRobinPartitionFunction extends PartitionFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + /** The current partition to use. */ + private int curPartition = 0; + + @Override + public TupleBatch[] partition(@Nonnull final TupleBatch tb) { + BitSet[] partitions = new BitSet[numPartition()]; + for (int i = 0; i < tb.numTuples(); i++) { + partitions[curPartition].set(i); + curPartition = (curPartition + 1) % numPartition(); + } + TupleBatch[] tbs = new TupleBatch[numPartition()]; + for (int i = 0; i < tbs.length; ++i) { + tbs[i] = tb.filter(partitions[i]); + } + return tbs; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/SinglePartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/SinglePartitionFunction.java new file mode 100644 index 000000000..f4db67be8 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/SinglePartitionFunction.java @@ -0,0 +1,26 @@ +package edu.washington.escience.myria.operator.network.distribute; + +import javax.annotation.Nonnull; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import edu.washington.escience.myria.storage.TupleBatch; + +/** return a fixed integer. */ +public final class SinglePartitionFunction extends PartitionFunction { + + /** Required for Java serialization. */ + private static final long serialVersionUID = 1L; + + /** + */ + @JsonCreator + public SinglePartitionFunction() { + setNumPartitions(1); + } + + @Override + public TupleBatch[] partition(@Nonnull final TupleBatch tb) { + return new TupleBatch[] {tb}; + } +} diff --git a/src/edu/washington/escience/myria/operator/network/distribute/package-info.java b/src/edu/washington/escience/myria/operator/network/distribute/package-info.java new file mode 100644 index 000000000..2db3249a2 --- /dev/null +++ b/src/edu/washington/escience/myria/operator/network/distribute/package-info.java @@ -0,0 +1,4 @@ +/** + * Functions that map tuples to machine partitions. + */ +package edu.washington.escience.myria.operator.network.distribute; diff --git a/src/edu/washington/escience/myria/operator/network/partition/FixValuePartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/FixValuePartitionFunction.java deleted file mode 100644 index bf2a70560..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/FixValuePartitionFunction.java +++ /dev/null @@ -1,39 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import javax.annotation.Nonnull; - -import edu.washington.escience.myria.storage.TupleBatch; - -/** - * - * return a fixed integer. - * - */ -public final class FixValuePartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * the fixed value this partition function returns. - */ - private final int value; - - /** - * @param value the fix value this partition function returns. - * - * */ - public FixValuePartitionFunction(final int value) { - super(1); - this.value = value; - } - - @Override - public int[] partition(@Nonnull final TupleBatch tb) { - final int[] result = new int[tb.numTuples()]; - for (int i = 0; i < result.length; i++) { - result[i] = value; - } - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/HowPartitioned.java b/src/edu/washington/escience/myria/operator/network/partition/HowPartitioned.java deleted file mode 100644 index e48db2a7b..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/HowPartitioned.java +++ /dev/null @@ -1,58 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import java.util.Set; - -import javax.annotation.Nullable; - -import jersey.repackaged.com.google.common.collect.ImmutableSet; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.primitives.Ints; - -/** - * How a dataset is partitioned. There are two fields being recorded for now, the partition function and the sequence of - * worker IDs. More information can be added in the future if needed. - */ -public class HowPartitioned { - - /** The partition function which was used to partition the dataset. Null means unknown. */ - @JsonProperty private PartitionFunction pf = null; - /** The sequence of workers that the dataset is partitioned on. Null means unknown. */ - @JsonProperty private ImmutableSet workers = null; - - /** - * @param pf the partition function. - * @param workers the sequence of workers. - */ - public HowPartitioned(@Nullable final PartitionFunction pf, @Nullable final int[] workers) { - this.pf = pf; - if (workers != null) { - this.workers = ImmutableSet.copyOf(Ints.asList(workers)); - } - } - - /** - * Static function to create a HowPartitioned object. - * - * @param pf the partition function. - * @param workers the sequence of workers. * - * @return a new HowPartitioned reference to the specified relation. - */ - @JsonCreator - public static HowPartitioned of( - @JsonProperty("pf") final PartitionFunction pf, - @JsonProperty("workers") final int[] workers) { - return new HowPartitioned(pf, workers); - } - - /** @return the partition function. */ - public PartitionFunction getPf() { - return pf; - } - - /** @return the workers. */ - public Set getWorkers() { - return workers; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/IdentityHashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/IdentityHashPartitionFunction.java deleted file mode 100644 index ff5f67eaa..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/IdentityHashPartitionFunction.java +++ /dev/null @@ -1,64 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import java.util.Objects; - -import javax.annotation.Nonnull; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.Type; -import edu.washington.escience.myria.storage.TupleBatch; - -/** - * Implementation of a PartitionFunction that use the trivial identity hash. - * (i.e. a --> a) The attribute to hash on must be an INT column and should - * represent a workerID - */ -public final class IdentityHashPartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** The index of the partition field. */ - @JsonProperty private final int index; - - /** - * @param index - * the index of the partition field. - */ - @JsonCreator - public IdentityHashPartitionFunction( - @JsonProperty(value = "index", required = true) final Integer index) { - super(null); - this.index = Objects.requireNonNull(index, "missing property index"); - Preconditions.checkArgument( - this.index >= 0, "IdentityHash field index cannot take negative value %s", this.index); - } - - /** - * @return the index - */ - public int getIndex() { - return index; - } - - /** - * @param tb - * data. - * @return partitions. - * */ - @Override - public int[] partition(@Nonnull final TupleBatch tb) { - Preconditions.checkArgument( - tb.getSchema().getColumnType(index) == Type.INT_TYPE, - "IdentityHash index column must be of type INT"); - final int[] result = new int[tb.numTuples()]; - for (int i = 0; i < result.length; i++) { - // Offset by -1 because WorkerIDs are 1-indexed. - result[i] = tb.getInt(index, i) - 1; - } - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/MFMDHashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/MFMDHashPartitionFunction.java deleted file mode 100644 index fb356972c..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/MFMDHashPartitionFunction.java +++ /dev/null @@ -1,65 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import java.util.Arrays; - -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.storage.TupleBatch; - -/** - * Multiple field multiple dimension hash partition function for HyperCubeJoin. - * - */ -public final class MFMDHashPartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * Partition functions on different dimensions. - */ - private final SingleFieldHashPartitionFunction[] partitionFunctions; - - /** - * - * @param numPartitions number of buckets - * @param hypercubeDimensions the sizes of each dimension of the hypercube. - * @param hashedColumns which fields are hashed. - * @param mappedHCDimensions mapped hypercube dimensions of hashed columns. - * - */ - public MFMDHashPartitionFunction( - final int numPartitions, - final int[] hypercubeDimensions, - final int[] hashedColumns, - final int[] mappedHCDimensions) { - super(numPartitions); - partitionFunctions = new SingleFieldHashPartitionFunction[hashedColumns.length]; - for (int i = 0; i < hashedColumns.length; ++i) { - Preconditions.checkPositionIndex(hashedColumns[i], hypercubeDimensions.length); - Preconditions.checkArgument( - hashedColumns.length == mappedHCDimensions.length, - "hashedColumns must have the same arity as mappedHCDimensions"); - partitionFunctions[i] = - new SingleFieldHashPartitionFunction( - hypercubeDimensions[mappedHCDimensions[i]], hashedColumns[i], mappedHCDimensions[i]); - } - } - - @Override - public int[] partition(final TupleBatch tb) { - int[] result = new int[tb.numTuples()]; - Arrays.fill(result, 0); - for (int i = 0; i < partitionFunctions.length; i++) { - int[] p = partitionFunctions[i].partition(tb); - for (int j = 0; j < tb.numTuples(); j++) { - result[j] = result[j] + p[j]; - if (i != partitionFunctions.length - 1) { - result[j] = result[j] * partitionFunctions[i + 1].numPartition(); - } - } - } - - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/MultiFieldHashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/MultiFieldHashPartitionFunction.java deleted file mode 100644 index fbd322e01..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/MultiFieldHashPartitionFunction.java +++ /dev/null @@ -1,85 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import java.util.Objects; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.storage.TupleBatch; -import edu.washington.escience.myria.util.HashUtils; - -/** - * Implementation that uses multiple fields as the key to hash - * - * The partition of a tuple is decided by the hash code of a group of fields of the tuple. - */ -public final class MultiFieldHashPartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** The indices used for partitioning. */ - @JsonProperty private final int[] indexes; - - /** - * @param numPartition number of partitions - * @param indexes the indices used for partitioning. - */ - @JsonCreator - public MultiFieldHashPartitionFunction( - @Nullable @JsonProperty("numPartitions") final Integer numPartition, - @JsonProperty(value = "indexes", required = true) final Integer[] indexes) { - super(numPartition); - Objects.requireNonNull(indexes, "indexes"); - Preconditions.checkArgument( - indexes.length > 1, "MultiFieldHash requires at least 2 fields to hash"); - this.indexes = new int[indexes.length]; - for (int i = 0; i < indexes.length; ++i) { - int index = - Preconditions.checkNotNull(indexes[i], "MultiFieldHash field index %s cannot be null", i); - Preconditions.checkArgument( - index >= 0, "MultiFieldHash field index %s cannot take negative value %s", i, index); - this.indexes[i] = index; - } - } - - /** - * @param numPartition number of partitions - * @param indexes the indices used for partitioning. - */ - public MultiFieldHashPartitionFunction(final Integer numPartition, final int[] indexes) { - super(numPartition); - Preconditions.checkArgument( - indexes.length > 1, "MultiFieldHash requires at least 2 fields to hash"); - this.indexes = indexes; - for (int i = 0; i < indexes.length; ++i) { - int index = indexes[i]; - Preconditions.checkArgument( - index >= 0, "MultiFieldHash field index %s cannot take negative value %s", i, index); - } - } - - /** - * @return the field indexes on which tuples will be hash partitioned. - */ - public int[] getIndexes() { - return indexes; - } - - @Override - public int[] partition(@Nonnull final TupleBatch tb) { - final int[] result = new int[tb.numTuples()]; - for (int i = 0; i < result.length; i++) { - int p = HashUtils.hashSubRow(tb, indexes, i) % numPartition(); - if (p < 0) { - p = p + numPartition(); - } - result[i] = p; - } - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/PartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/PartitionFunction.java deleted file mode 100644 index 1fbb98b0f..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/PartitionFunction.java +++ /dev/null @@ -1,80 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import java.io.Serializable; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonSubTypes.Type; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.storage.TupleBatch; - -/** - * The ShuffleProducer class uses an instance of the PartitionFunction class to decide which worker a tuple should be - * routed to. Typically, the ShuffleProducer class invokes {@link partition(Tuple, Schema) partition} on every tuple it - * generates. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") -@JsonSubTypes({ - @Type(value = RoundRobinPartitionFunction.class, name = "RoundRobin"), - @Type(value = SingleFieldHashPartitionFunction.class, name = "SingleFieldHash"), - @Type(value = IdentityHashPartitionFunction.class, name = "IdentityHash"), - @Type(value = MultiFieldHashPartitionFunction.class, name = "MultiFieldHash"), - @Type(value = WholeTupleHashPartitionFunction.class, name = "WholeTupleHash") -}) -public abstract class PartitionFunction implements Serializable { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * The number of partitions into which input tuples can be divided. - */ - @JsonProperty private Integer numPartitions = null; - - /** - * @param numPartitions the number of partitions into which input tuples can be divided. Note that this is a - * {@link Integer} not an {@link int} so that it can properly handle null values, e.g., in JSON - * deserialization. - */ - public PartitionFunction(@Nullable final Integer numPartitions) { - Preconditions.checkArgument( - (numPartitions == null) || (numPartitions > 0), - "numPartitions argument must be null or > 0"); - this.numPartitions = numPartitions; - } - - /** - * @return the number of partitions. - */ - public final int numPartition() { - Preconditions.checkState(numPartitions != null, "numPartitions has not been set"); - return numPartitions; - } - - /** - * Given that the TupleBatches expose only the valid tuples, partition functions using TB.get** methods should be of - * little overhead comparing with direct Column access. - * - * @param data the data to be partitioned. - * - * @return an int[] of length specified by data.{@link TupleBatch#numTuples}, specifying which partition - * every tuple should be sent to. - * - */ - public abstract int[] partition(@Nonnull final TupleBatch data); - - /** - * Set the number of output partitions. - * - * @param numPartitions the number of output partitions. Must be greater than 0. - */ - public final void setNumPartitions(final int numPartitions) { - Preconditions.checkArgument(numPartitions > 0, "numPartitions must be > 0"); - this.numPartitions = numPartitions; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/RoundRobinPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/RoundRobinPartitionFunction.java deleted file mode 100644 index 1b4c71cc4..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/RoundRobinPartitionFunction.java +++ /dev/null @@ -1,41 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import edu.washington.escience.myria.storage.TupleBatch; - -/** - * A partition function that simply sends one tuple to each output in turn. - * - * - */ -public final class RoundRobinPartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - /** The next partition to use. */ - private int partition = 0; - - /** - * @param numPartitions the number of partitions. - */ - @JsonCreator - public RoundRobinPartitionFunction( - @Nullable @JsonProperty("numPartitions") final Integer numPartitions) { - super(numPartitions); - } - - @Override - public int[] partition(@Nonnull final TupleBatch tb) { - final int[] result = new int[tb.numTuples()]; - for (int i = 0; i < result.length; i++) { - result[i] = partition; - partition = (partition + 1) % numPartition(); - } - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/SingleFieldHashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/SingleFieldHashPartitionFunction.java deleted file mode 100644 index c74d25420..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/SingleFieldHashPartitionFunction.java +++ /dev/null @@ -1,82 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import javax.annotation.Nonnull; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.MoreObjects; -import com.google.common.base.Preconditions; - -import edu.washington.escience.myria.storage.TupleBatch; -import edu.washington.escience.myria.util.HashUtils; - -/** - * The default implementation of the partition function. - * - * The partition of a tuple is decided by the hash code of a preset field of the tuple. - */ -public final class SingleFieldHashPartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * The index of the partition field. - */ - @JsonProperty private final int index; - - /** - * The index of the chosen hashcode in HashUtils. - */ - @JsonProperty private final int seedIndex; - - /** - * @param numPartitions number of partitions. - * @param index the index of the partition field. - * @param seedIndex the index of chosen hash seed. - */ - @JsonCreator - public SingleFieldHashPartitionFunction( - @JsonProperty("numPartitions") final Integer numPartitions, - @JsonProperty(value = "index", required = true) final Integer index, - @JsonProperty(value = "seedIndex") final Integer seedIndex) { - super(numPartitions); - /* TODO(dhalperi) once Jackson actually implements support for required, remove these checks. */ - this.index = java.util.Objects.requireNonNull(index, "missing property index"); - this.seedIndex = MoreObjects.firstNonNull(seedIndex, 0) % HashUtils.NUM_OF_HASHFUNCTIONS; - Preconditions.checkArgument( - this.index >= 0, "SingleFieldHash field index cannot take negative value %s", this.index); - } - - /** - * @param numPartitions numPartitions number of partitions. - * @param index the index of the partition field. - */ - public SingleFieldHashPartitionFunction(final Integer numPartitions, final Integer index) { - this(numPartitions, index, Integer.valueOf(0)); - } - - /** - * @return the index - */ - public int getIndex() { - return index; - } - - /** - * @param tb data. - * @return partitions. - * */ - @Override - public int[] partition(final @Nonnull TupleBatch tb) { - final int[] result = new int[tb.numTuples()]; - for (int i = 0; i < result.length; i++) { - int p = HashUtils.hashValue(tb, index, i, seedIndex) % numPartition(); - if (p < 0) { - p = p + numPartition(); - } - result[i] = p; - } - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/WholeTupleHashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/partition/WholeTupleHashPartitionFunction.java deleted file mode 100644 index b0b5d79c7..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/WholeTupleHashPartitionFunction.java +++ /dev/null @@ -1,41 +0,0 @@ -package edu.washington.escience.myria.operator.network.partition; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import edu.washington.escience.myria.storage.TupleBatch; -import edu.washington.escience.myria.util.HashUtils; - -/** - * Partition of tuples by the hash code of the whole tuple. - */ -public final class WholeTupleHashPartitionFunction extends PartitionFunction { - - /** Required for Java serialization. */ - private static final long serialVersionUID = 1L; - - /** - * @param numPartitions total number of partitions. - */ - @JsonCreator - public WholeTupleHashPartitionFunction( - @Nullable @JsonProperty("numPartitions") final Integer numPartitions) { - super(numPartitions); - } - - @Override - public int[] partition(@Nonnull final TupleBatch tb) { - final int[] result = new int[tb.numTuples()]; - for (int i = 0; i < result.length; i++) { - int p = HashUtils.hashRow(tb, i) % numPartition(); - if (p < 0) { - p = p + numPartition(); - } - result[i] = p; - } - return result; - } -} diff --git a/src/edu/washington/escience/myria/operator/network/partition/package-info.java b/src/edu/washington/escience/myria/operator/network/partition/package-info.java deleted file mode 100644 index 7013fbef1..000000000 --- a/src/edu/washington/escience/myria/operator/network/partition/package-info.java +++ /dev/null @@ -1,4 +0,0 @@ -/** - * Functions that map tuples to machine partitions. - */ -package edu.washington.escience.myria.operator.network.partition; diff --git a/src/edu/washington/escience/myria/parallel/RelationWriteMetadata.java b/src/edu/washington/escience/myria/parallel/RelationWriteMetadata.java index b2909e31f..08e35e9ed 100644 --- a/src/edu/washington/escience/myria/parallel/RelationWriteMetadata.java +++ b/src/edu/washington/escience/myria/parallel/RelationWriteMetadata.java @@ -12,11 +12,9 @@ import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.Schema; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; -/** - * Holds metadata about a relation that is created by a subquery. - */ +/** Holds metadata about a relation that is created by a subquery. */ public class RelationWriteMetadata implements Serializable { /** Required for Java serialization. */ private static final long serialVersionUID = 1L; @@ -30,19 +28,17 @@ public class RelationWriteMetadata implements Serializable { private final boolean overwrite; /** Whether the relation being written is a temporary or a permanent relation. */ private final boolean temporary; - /** The PartitionFunction used to partition the relation across workers. */ - private final PartitionFunction partitionFunction; + /** The DistributeFunction used to distribute the relation across workers. */ + private final DistributeFunction distributeFunction; - /** - * Constructs a new relation metadata object. + /** Constructs a new relation metadata object. * * @param relationKey the relation to be written * @param schema the schema of the write. * @param overwrite if {@code true}, then the relation will be overwritten / created. if false, the relation will be - * created or appended. If appending, the schema must match the catalog schema. + * created or appended. If appending, the schema must match the catalog schema. * @param temporary if {@code true}, then the relation will be not be added to the Catalog, and its tuple count will - * not be maintained. - */ + * not be maintained. */ public RelationWriteMetadata( @Nonnull final RelationKey relationKey, @Nonnull final Schema schema, @@ -51,95 +47,77 @@ public RelationWriteMetadata( this(relationKey, schema, overwrite, temporary, null); } - /** - * Constructs a new relation metadata object. + /** Constructs a new relation metadata object. * * @param relationKey the relation to be written * @param schema the schema of the write. * @param overwrite if {@code true}, then the relation will be overwritten / created. if false, the relation will be - * created or appended. If appending, the schema must match the catalog schema. + * created or appended. If appending, the schema must match the catalog schema. * @param temporary if {@code true}, then the relation will be not be added to the Catalog, and its tuple count will - * not be maintained. - * @param partitionFunction the PartitionFunction used to partition the relation across workers. - */ + * not be maintained. + * @param distributeFunction the DistributeFunction used to distribute the relation across workers. */ public RelationWriteMetadata( @Nonnull final RelationKey relationKey, @Nonnull final Schema schema, final boolean overwrite, final boolean temporary, - @Nullable final PartitionFunction partitionFunction) { + @Nullable final DistributeFunction distributeFunction) { this.relationKey = Objects.requireNonNull(relationKey, "relationKey"); this.schema = Objects.requireNonNull(schema, "schema"); this.overwrite = overwrite; this.temporary = temporary; - this.partitionFunction = partitionFunction; + this.distributeFunction = distributeFunction; workers = new HashSet<>(); } - /** - * Add the specified worker to set of workers writing this relation. + /** Add the specified worker to set of workers writing this relation. * - * @param workerId the id of the worker - */ + * @param workerId the id of the worker */ public void addWorker(final Integer workerId) { workers.add(workerId); } - /** - * Indicates whether the relation will be overwritten if it already exists. If {@code false} and the relation does + /** Indicates whether the relation will be overwritten if it already exists. If {@code false} and the relation does * already exist, tuples will be appended to the relation and the schema of these tuples must match the schema already * in the catalog. * - * @return {@code true} if the relation will be overwritten if it already exists, or {@code false} otherwise. - */ + * @return {@code true} if the relation will be overwritten if it already exists, or {@code false} otherwise. */ public boolean isOverwrite() { return overwrite; } - /** - * Indicates whether the relation is a temporary relation ({@code true}) or will be persisted ({@code false}). If ( + /** Indicates whether the relation is a temporary relation ({@code true}) or will be persisted ({@code false}). If ( * {@code false}), the relation will not be persisted and/or added to the catalog, reducing query overhead. * - * @return if {@code false}, the relation will be persisted and entered into the catalog. - */ + * @return if {@code false}, the relation will be persisted and entered into the catalog. */ public boolean isTemporary() { return temporary; } - /** - * Get the key of the relation to be written. + /** Get the key of the relation to be written. * - * @return the key of the relation to be written. - */ + * @return the key of the relation to be written. */ public RelationKey getRelationKey() { return relationKey; } - /** - * Get the schema of the tuples to be written to this relation. + /** Get the schema of the tuples to be written to this relation. * - * @return the schema of the tuples to be written to this relation - */ + * @return the schema of the tuples to be written to this relation */ public Schema getSchema() { return schema; } - /** - * Get the set of workers on which this relation will be written. + /** Get the set of workers on which this relation will be written. * - * @return the set of workers on which this relation will be written. - */ + * @return the set of workers on which this relation will be written. */ public Set getWorkers() { return ImmutableSet.copyOf(workers); } - /** - * Get the PartitionFunction used to partition this relation. - * - * @return the PartitionFunction used to partition this relation - */ - public PartitionFunction getPartitionFunction() { - return partitionFunction; + /** @return the DistributeFunction used to distribute this relation */ + public DistributeFunction getDistributeFunction() { + return distributeFunction; } @Override diff --git a/src/edu/washington/escience/myria/parallel/Server.java b/src/edu/washington/escience/myria/parallel/Server.java index f3562c764..717e25b6f 100644 --- a/src/edu/washington/escience/myria/parallel/Server.java +++ b/src/edu/washington/escience/myria/parallel/Server.java @@ -4,6 +4,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.ObjectInputStream; +import java.net.BindException; import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Arrays; @@ -38,7 +39,6 @@ import org.apache.reef.tang.Injector; import org.apache.reef.tang.Tang; import org.apache.reef.tang.annotations.Parameter; -import org.apache.reef.tang.exceptions.BindException; import org.apache.reef.tang.exceptions.InjectionException; import org.apache.reef.tang.formats.AvroConfigurationSerializer; import org.apache.reef.tang.formats.ConfigurationSerializer; @@ -64,6 +64,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.Striped; import com.google.protobuf.InvalidProtocolBufferException; @@ -109,12 +110,11 @@ import edu.washington.escience.myria.operator.agg.PrimitiveAggregator.AggregationOp; import edu.washington.escience.myria.operator.agg.SingleColumnAggregatorFactory; import edu.washington.escience.myria.operator.agg.SingleGroupByAggregate; -import edu.washington.escience.myria.operator.network.CollectConsumer; import edu.washington.escience.myria.operator.network.CollectProducer; -import edu.washington.escience.myria.operator.network.GenericShuffleConsumer; +import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; -import edu.washington.escience.myria.operator.network.partition.HowPartitioned; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.HowDistributed; import edu.washington.escience.myria.parallel.ipc.IPCConnectionPool; import edu.washington.escience.myria.parallel.ipc.IPCMessage; import edu.washington.escience.myria.parallel.ipc.InJVMLoopbackChannelSink; @@ -145,14 +145,10 @@ import edu.washington.escience.myria.util.concurrent.ErrorLoggingTimerTask; import edu.washington.escience.myria.util.concurrent.RenamingThreadFactory; -/** - * The master entrance. - */ +/** The master entrance. */ public final class Server implements TaskMessageSource, EventHandler { - /** - * Master message processor. - */ + /** Master message processor. */ private final class MessageProcessor implements Runnable { /** Constructor, set the thread name. */ @@ -254,10 +250,8 @@ private void enqueueDriverMessage(@Nonnull final TransportMessage msg) { /* * (non-Javadoc) - * - * @see org.apache.reef.task.TaskMessageSource#getMessage() - * - * To be used to instruct the driver to launch or abort workers. + * @see org.apache.reef.task.TaskMessageSource#getMessage() To be used to instruct the driver to launch or abort + * workers. */ @Override public Optional getMessage() { @@ -267,9 +261,7 @@ public Optional getMessage() { private Striped workerAddRemoveLock; - /** - * REEF event handler for driver messages indicating worker failure. - */ + /** REEF event handler for driver messages indicating worker failure. */ @Override public void onNext(final DriverMessage driverMessage) { LOGGER.info("Driver message received"); @@ -341,94 +333,64 @@ public void operationComplete(final ChannelGroupFuture future) { /** The logger for this class. */ private static final org.slf4j.Logger LOGGER = LoggerFactory.getLogger(Server.class); - /** - * Initial worker list. - */ + /** Initial worker list. */ private ImmutableMap workers = null; /** Manages the queries executing in this instance of Myria. */ private QueryManager queryManager = null; - /** - * @return the query manager. - */ + /** @return the query manager. */ public QueryManager getQueryManager() { return queryManager; } - /** - * Current alive worker set. - */ + /** Current alive worker set. */ private final Set aliveWorkers; - /** - * Execution environment variables for operators. - */ + /** Execution environment variables for operators. */ private final ConcurrentHashMap execEnvVars; - /** - * All message queue. + /** All message queue. * - * @TODO remove this queue as in {@link Worker}s. - */ + * @TODO remove this queue as in {@link Worker}s. */ private final LinkedBlockingQueue> messageQueue; - /** - * The IPC Connection Pool. - */ + /** The IPC Connection Pool. */ private IPCConnectionPool connectionPool; - /** - * {@link ExecutorService} for message processing. - */ + /** {@link ExecutorService} for message processing. */ private volatile ExecutorService messageProcessingExecutor; /** The Catalog stores the metadata about the Myria instance. */ private MasterCatalog catalog; - /** - * The {@link OrderedMemoryAwareThreadPoolExecutor} who gets messages from {@link workerExecutor} and further process - * them using application specific message handlers, e.g. {@link MasterShortMessageProcessor}. - */ + /** The {@link OrderedMemoryAwareThreadPoolExecutor} who gets messages from {@link workerExecutor} and further process + * them using application specific message handlers, e.g. {@link MasterShortMessageProcessor}. */ private volatile OrderedMemoryAwareThreadPoolExecutor ipcPipelineExecutor; - /** - * The {@link ExecutorService} who executes the master-side subqueries. - */ + /** The {@link ExecutorService} who executes the master-side subqueries. */ private volatile ExecutorService serverQueryExecutor; - /** - * Absolute path of the directory containing the master catalog files - */ + /** Absolute path of the directory containing the master catalog files */ private final String catalogPath; - /** - * The URI to persist relations - */ + /** The URI to persist relations */ private final String persistURI; - /** - * @return the query executor used in this worker. - */ + /** @return the query executor used in this worker. */ ExecutorService getQueryExecutor() { return serverQueryExecutor; } - /** - * max number of seconds for elegant cleanup. - */ + /** max number of seconds for elegant cleanup. */ public static final int NUM_SECONDS_FOR_ELEGANT_CLEANUP = 10; - /** - * @return my connection pool for IPC. - */ + /** @return my connection pool for IPC. */ IPCConnectionPool getIPCConnectionPool() { return connectionPool; } - /** - * @return my pipeline executor. - */ + /** @return my pipeline executor. */ OrderedMemoryAwareThreadPoolExecutor getPipelineExecutor() { return ipcPipelineExecutor; } @@ -436,16 +398,12 @@ OrderedMemoryAwareThreadPoolExecutor getPipelineExecutor() { /** The socket info for the master. */ private final SocketInfo masterSocketInfo; - /** - * @return my execution environment variables for init of operators. - */ + /** @return my execution environment variables for init of operators. */ ConcurrentHashMap getExecEnvVars() { return execEnvVars; } - /** - * @return execution mode. - */ + /** @return execution mode. */ QueryExecutionMode getExecutionMode() { return QueryExecutionMode.NON_BLOCKING; } @@ -459,8 +417,7 @@ QueryExecutionMode getExecutionMode() { private final int inputBufferRecoverTrigger; private final Injector injector; - /** - * Construct a server object, with configuration stored in the specified catalog file. + /** Construct a server object, with configuration stored in the specified catalog file. * * @param masterHost hostname of the master * @param masterPort RPC port of the master @@ -474,8 +431,7 @@ QueryExecutionMode getExecutionMode() { * @param inputBufferCapacity size of the input buffer in bytes * @param inputBufferRecoverTrigger number of bytes in the input buffer to trigger recovery after overflow * @param persistURI the storage endpoint URI for persisting partitioned relations - * @param injector a Tang injector for instantiating objects from configuration - */ + * @param injector a Tang injector for instantiating objects from configuration */ @Inject public Server( @Parameter(MasterHost.class) final String masterHost, @@ -514,19 +470,13 @@ public Server( messageQueue = new LinkedBlockingQueue<>(); } - /** - * timer task executor. - */ + /** timer task executor. */ private ScheduledExecutorService scheduledTaskExecutor; - /** - * This class presents only for the purpose of debugging. No other usage. - */ + /** This class presents only for the purpose of debugging. No other usage. */ private class DebugHelper extends ErrorLoggingTimerTask { - /** - * Interval of execution. - */ + /** Interval of execution. */ public static final int INTERVAL = MyriaConstants.WAITING_INTERVAL_1_SECOND_IN_MS; @Override @@ -565,9 +515,7 @@ private static Integer getPortFromWorkerConf(final Configuration workerConf) return injector.getNamedInstance(MyriaWorkerConfigurationModule.WorkerPort.class); } - /** - * Master cleanup. - */ + /** Master cleanup. */ private void cleanup() { LOGGER.info("{} is going to shutdown", MyriaConstants.SYSTEM_NAME); @@ -596,18 +544,14 @@ private void cleanup() { LOGGER.info("Master finishes cleanup."); } - /** - * Shutdown the master. - */ + /** Shutdown the master. */ public void shutdown() { cleanup(); } - /** - * Start all the threads that do work for the server. + /** Start all the threads that do work for the server. * - * @throws Exception if any error occurs. - */ + * @throws Exception if any error occurs. */ public void start() throws Exception { LOGGER.info("Server starting on {}", masterSocketInfo); @@ -668,33 +612,28 @@ public void start() throws Exception { serverQueryExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master query executor")); - /** - * The {@link Executor} who deals with IPC connection setup/cleanup. - */ + /** The {@link Executor} who deals with IPC connection setup/cleanup. */ ExecutorService ipcBossExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master IPC boss")); - /** - * The {@link Executor} who deals with IPC message delivering and transformation. - */ + /** The {@link Executor} who deals with IPC message delivering and transformation. */ ExecutorService ipcWorkerExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master IPC worker")); ipcPipelineExecutor = null; // Remove the pipeline executor. - // new OrderedMemoryAwareThreadPoolExecutor(Runtime.getRuntime().availableProcessors() * 2 + 1, - // 5 * MyriaConstants.MB, 0, MyriaConstants.THREAD_POOL_KEEP_ALIVE_TIME_IN_MS, + // new + // OrderedMemoryAwareThreadPoolExecutor(Runtime.getRuntime().availableProcessors() + // * 2 + 1, + // 5 * MyriaConstants.MB, 0, + // MyriaConstants.THREAD_POOL_KEEP_ALIVE_TIME_IN_MS, // TimeUnit.MILLISECONDS, // new RenamingThreadFactory("Master Pipeline executor")); - /** - * The {@link ChannelFactory} for creating client side connections. - */ + /** The {@link ChannelFactory} for creating client side connections. */ ChannelFactory clientChannelFactory = new NioClientSocketChannelFactory( ipcBossExecutor, ipcWorkerExecutor, Runtime.getRuntime().availableProcessors() * 2 + 1); - /** - * The {@link ChannelFactory} for creating server side accepted connections. - */ + /** The {@link ChannelFactory} for creating server side accepted connections. */ ChannelFactory serverChannelFactory = new NioServerSocketChannelFactory( ipcBossExecutor, ipcWorkerExecutor, Runtime.getRuntime().availableProcessors() * 2 + 1); @@ -719,7 +658,7 @@ public void start() throws Exception { LOGGER.info("Server started on {}", masterSocketInfo); if (getDBMS().equals(MyriaConstants.STORAGE_SYSTEM_POSTGRESQL)) { - final Set workerIds = workers.keySet(); + final List workerIds = ImmutableList.copyOf(workers.keySet()); addRelationToCatalog( MyriaConstants.EVENT_PROFILING_RELATION, MyriaConstants.EVENT_PROFILING_SCHEMA, @@ -738,20 +677,17 @@ public void start() throws Exception { } } - /** - * Manually add a relation to the catalog. + /** Manually add a relation to the catalog. * * @param relationKey the relation to add * @param schema the schema of the relation to add * @param workers the workers that have the relation * @param force force add the relation; will replace an existing entry. - * - * @throws DbException if the catalog cannot be accessed - */ + * @throws DbException if the catalog cannot be accessed */ private void addRelationToCatalog( final RelationKey relationKey, final Schema schema, - final Set workers, + final List workers, final boolean force) throws DbException { try { @@ -790,23 +726,18 @@ private void addRelationToCatalog( } } - /** - * @return the dbms from {@link #execEnvVars}. - */ + /** @return the dbms from {@link #execEnvVars}. */ public String getDBMS() { return (String) execEnvVars.get(MyriaConstants.EXEC_ENV_VAR_DATABASE_SYSTEM); } - /** - * - * Can be only used in test. + /** Can be only used in test. * * @return true if the query plan is accepted and scheduled for execution. * @param masterRoot the root operator of the master plan * @param workerRoots the roots of the worker part of the plan, {workerID -> RootOperator[]} * @throws DbException if any error occurs. - * @throws CatalogException catalog errors. - */ + * @throws CatalogException catalog errors. */ public QueryFuture submitQueryPlan( final RootOperator masterRoot, final Map workerRoots) throws DbException, CatalogException { @@ -823,19 +754,15 @@ public QueryFuture submitQueryPlan( workerPlans); } - /** - * @return the set of workers that are currently alive. - */ + /** @return the set of workers that are currently alive. */ public Set getAliveWorkers() { return ImmutableSet.copyOf(aliveWorkers); } - /** - * Return a random subset of workers. + /** Return a random subset of workers. * * @param number the number of alive workers returned - * @return a subset of workers that are currently alive. - */ + * @return a subset of workers that are currently alive. */ public Set getRandomWorkers(final int number) { Preconditions.checkArgument( number <= getAliveWorkers().size(), @@ -848,50 +775,44 @@ public Set getRandomWorkers(final int number) { return ImmutableSet.copyOf(workerList.subList(0, number)); } - /** - * @return the set of known workers in this Master. - */ + /** @return the set of known workers in this Master. */ public Map getWorkers() { return workers; } - /** - * Ingest the given dataset. + /** Ingest the given dataset. * * @param relationKey the name of the dataset. * @param workersToIngest restrict the workers to ingest data (null for all) * @param indexes the indexes created. * @param source the source of tuples to be ingested. - * @param pf the PartitionFunction used to partition the ingested relation. + * @param df the distribute function. * @return the status of the ingested dataset. * @throws InterruptedException interrupted - * @throws DbException if there is an error - */ + * @throws DbException if there is an error */ public DatasetStatus ingestDataset( final RelationKey relationKey, - final Set workersToIngest, + List workersToIngest, final List> indexes, final Operator source, - final PartitionFunction pf) + final DistributeFunction df) throws InterruptedException, DbException { /* Figure out the workers we will use. If workersToIngest is null, use all active workers. */ - Set actualWorkers = workersToIngest; if (workersToIngest == null) { - actualWorkers = getAliveWorkers(); + workersToIngest = ImmutableList.copyOf(getAliveWorkers()); } - Preconditions.checkArgument(actualWorkers.size() > 0, "Must use > 0 workers"); - int[] workersArray = MyriaUtils.integerSetToIntArray(actualWorkers); + int[] workersArray = Ints.toArray(workersToIngest); + Preconditions.checkArgument(workersArray.length > 0, "Must use > 0 workers"); /* The master plan: send the tuples out. */ ExchangePairID scatterId = ExchangePairID.newID(); - pf.setNumPartitions(workersArray.length); + df.setNumDestinations(workersArray.length, 1); GenericShuffleProducer scatter = - new GenericShuffleProducer(source, scatterId, workersArray, pf); + new GenericShuffleProducer(source, new ExchangePairID[] {scatterId}, workersArray, df); /* The workers' plan */ - GenericShuffleConsumer gather = - new GenericShuffleConsumer( - source.getSchema(), scatterId, new int[] {MyriaConstants.MASTER_ID}); + Consumer gather = + new Consumer(source.getSchema(), scatterId, ImmutableSet.of(MyriaConstants.MASTER_ID)); DbInsert insert = new DbInsert(gather, relationKey, true, indexes); Map workerPlans = new HashMap<>(); for (Integer workerId : workersArray) { @@ -917,19 +838,17 @@ public DatasetStatus ingestDataset( } // updating the partition function only after it's successfully ingested. - updateHowPartitioned(relationKey, new HowPartitioned(pf, workersArray)); + updateHowPartitioned(relationKey, new HowDistributed(df, workersArray)); return getDatasetStatus(relationKey); } - /** - * Parallel Ingest + /** Parallel Ingest * * @param relationKey the name of the dataset. * @param workersToIngest restrict the workers to ingest data (null for all) * @throws URIException * @throws DbException - * @throws InterruptedException - */ + * @throws InterruptedException */ public DatasetStatus parallelIngestDataset( final RelationKey relationKey, final Schema schema, @@ -939,7 +858,7 @@ public DatasetStatus parallelIngestDataset( @Nullable final Integer numberOfSkippedLines, final AmazonS3Source s3Source, final Set workersToIngest, - final PartitionFunction partitionFunction) + final DistributeFunction distributeFunction) throws URIException, DbException, InterruptedException { /* Figure out the workers we will use */ Set actualWorkers = workersToIngest; @@ -1013,27 +932,26 @@ public DatasetStatus parallelIngestDataset( throw new DbException("Error executing query", e.getCause()); } - updateHowPartitioned(relationKey, new HowPartitioned(partitionFunction, workersArray)); + updateHowPartitioned(relationKey, new HowDistributed(distributeFunction, workersArray)); return getDatasetStatus(relationKey); } - /** - * @param relationKey the relationalKey of the dataset to import + /** @param relationKey the relationalKey of the dataset to import * @param schema the schema of the dataset to import * @param workersToImportFrom the set of workers * @throws DbException if there is an error - * @throws InterruptedException interrupted - */ + * @throws InterruptedException interrupted */ public void addDatasetToCatalog( - final RelationKey relationKey, final Schema schema, final Set workersToImportFrom) + final RelationKey relationKey, final Schema schema, final List workersToImportFrom) throws DbException, InterruptedException { - /* Figure out the workers we will use. If workersToImportFrom is null, use all active workers. */ - Set actualWorkers = workersToImportFrom; + /* + * Figure out the workers we will use. If workersToImportFrom is null, use all active workers. + */ + List actualWorkers = workersToImportFrom; if (workersToImportFrom == null) { - actualWorkers = getWorkers().keySet(); + actualWorkers = ImmutableList.copyOf(getWorkers().keySet()); } - addRelationToCatalog(relationKey, schema, workersToImportFrom, true); try { @@ -1059,12 +977,10 @@ public void addDatasetToCatalog( } } - /** - * @param relationKey the relationKey of the dataset to delete + /** @param relationKey the relationKey of the dataset to delete * @return the status * @throws DbException if there is an error - * @throws InterruptedException interrupted - */ + * @throws InterruptedException interrupted */ public void deleteDataset(final RelationKey relationKey) throws DbException, InterruptedException { @@ -1108,9 +1024,7 @@ public void deleteDataset(final RelationKey relationKey) } } - /** - * Create indexes and add the metadata to the catalog - */ + /** Create indexes and add the metadata to the catalog */ public long addIndexesToRelation( final RelationKey relationKey, final Schema schema, final List indexes) throws DbException, InterruptedException { @@ -1155,9 +1069,7 @@ public long addIndexesToRelation( return queryID; } - /** - * Create a view - */ + /** Create a view */ public long createView( final String viewName, final String viewDefinition, final Set workers) throws DbException, InterruptedException { @@ -1196,14 +1108,12 @@ public long createView( return queryID; } - /** - * Create a function and register it in the catalog + /** Create a function and register it in the catalog * * @param name the name of the function * @param definition the function definition (must be postgres specific) * @param outputSchema the output schema of the function - * @return the status of the function - */ + * @return the status of the function */ public String createFunction( final String name, final String definition, @@ -1263,12 +1173,10 @@ public String createFunction( return response; } - /** - * @param relationKey the relationKey of the dataset to persist + /** @param relationKey the relationKey of the dataset to persist * @return the queryID * @throws DbException if there is an error - * @throws InterruptedException interrupted - */ + * @throws InterruptedException interrupted */ public long persistDataset(final RelationKey relationKey) throws DbException, InterruptedException, URISyntaxException { long queryID; @@ -1319,21 +1227,17 @@ public long persistDataset(final RelationKey relationKey) return queryID; } - /** - * @param relationKey the key of the desired relation. + /** @param relationKey the key of the desired relation. * @return the schema of the specified relation, or null if not found. - * @throws CatalogException if there is an error getting the Schema out of the catalog. - */ + * @throws CatalogException if there is an error getting the Schema out of the catalog. */ public Schema getSchema(final RelationKey relationKey) throws CatalogException { return catalog.getSchema(relationKey); } - /** - * @param key the relation key. + /** @param key the relation key. * @param howPartitioned how the dataset was partitioned. - * @throws DbException if there is an catalog exception. - */ - public void updateHowPartitioned(final RelationKey key, final HowPartitioned howPartitioned) + * @throws DbException if there is an catalog exception. */ + public void updateHowPartitioned(final RelationKey key, final HowDistributed howPartitioned) throws DbException { try { catalog.updateHowPartitioned(key, howPartitioned); @@ -1342,38 +1246,30 @@ public void updateHowPartitioned(final RelationKey key, final HowPartitioned how } } - /** - * @param relationKey the key of the desired relation. + /** @param relationKey the key of the desired relation. * @param storedRelationId indicates which copy of the desired relation we want to scan. * @return the list of workers that store the specified relation. - * @throws CatalogException if there is an error accessing the catalog. - */ + * @throws CatalogException if there is an error accessing the catalog. */ public Set getWorkersForRelation( final RelationKey relationKey, final Integer storedRelationId) throws CatalogException { return catalog.getWorkersForRelation(relationKey, storedRelationId); } - /** - * @param queryId the query that owns the desired temp relation. + /** @param queryId the query that owns the desired temp relation. * @param relationKey the key of the desired temp relation. - * @return the list of workers that store the specified relation. - */ + * @return the list of workers that store the specified relation. */ public Set getWorkersForTempRelation( @Nonnull final Long queryId, @Nonnull final RelationKey relationKey) { return queryManager.getQuery(queryId).getWorkersForTempRelation(relationKey); } - /** - * @return the socket info for the master. - */ + /** @return the socket info for the master. */ protected SocketInfo getSocketInfo() { return masterSocketInfo; } - /** - * @return A list of datasets in the system. - * @throws DbException if there is an error accessing the desired Schema. - */ + /** @return A list of datasets in the system. + * @throws DbException if there is an error accessing the desired Schema. */ public List getDatasets() throws DbException { try { return catalog.getDatasets(); @@ -1382,13 +1278,11 @@ public List getDatasets() throws DbException { } } - /** - * Get the metadata about a relation. + /** Get the metadata about a relation. * * @param relationKey specified which relation to get the metadata about. * @return the metadata of the specified relation. - * @throws DbException if there is an error getting the status. - */ + * @throws DbException if there is an error getting the status. */ public DatasetStatus getDatasetStatus(final RelationKey relationKey) throws DbException { try { return catalog.getDatasetStatus(relationKey); @@ -1397,11 +1291,9 @@ public DatasetStatus getDatasetStatus(final RelationKey relationKey) throws DbEx } } - /** - * @param searchTerm the search term + /** @param searchTerm the search term * @return the relations that match the search term - * @throws DbException if there is an error getting the relation keys. - */ + * @throws DbException if there is an error getting the relation keys. */ public List getMatchingRelationKeys(final String searchTerm) throws DbException { try { return catalog.getMatchingRelationKeys(searchTerm); @@ -1410,11 +1302,9 @@ public List getMatchingRelationKeys(final String searchTerm) throws } } - /** - * @param userName the user whose datasets we want to access. + /** @param userName the user whose datasets we want to access. * @return a list of datasets belonging to the specified user. - * @throws DbException if there is an error accessing the Catalog. - */ + * @throws DbException if there is an error accessing the Catalog. */ public List getDatasetsForUser(final String userName) throws DbException { try { return catalog.getDatasetsForUser(userName); @@ -1423,12 +1313,10 @@ public List getDatasetsForUser(final String userName) throws DbEx } } - /** - * @param userName the user whose datasets we want to access. + /** @param userName the user whose datasets we want to access. * @param programName the program by that user whose datasets we want to access. * @return a list of datasets belonging to the specified program. - * @throws DbException if there is an error accessing the Catalog. - */ + * @throws DbException if there is an error accessing the Catalog. */ public List getDatasetsForProgram(final String userName, final String programName) throws DbException { try { @@ -1438,11 +1326,9 @@ public List getDatasetsForProgram(final String userName, final St } } - /** - * @param queryId the id of the query. + /** @param queryId the id of the query. * @return a list of datasets belonging to the specified program. - * @throws DbException if there is an error accessing the Catalog. - */ + * @throws DbException if there is an error accessing the Catalog. */ public List getDatasetsForQuery(final int queryId) throws DbException { try { return catalog.getDatasetsForQuery(queryId); @@ -1451,33 +1337,27 @@ public List getDatasetsForQuery(final int queryId) throws DbExcep } } - /** - * @return the maximum query id that matches the search. + /** @return the maximum query id that matches the search. * @param searchTerm a token to match against the raw queries. If null, all queries match. - * @throws CatalogException if an error occurs - */ + * @throws CatalogException if an error occurs */ public long getMaxQuery(final String searchTerm) throws CatalogException { return catalog.getMaxQuery(searchTerm); } - /** - * @return the minimum query id that matches the search. + /** @return the minimum query id that matches the search. * @param searchTerm a token to match against the raw queries. If null, all queries match. - * @throws CatalogException if an error occurs - */ + * @throws CatalogException if an error occurs */ public long getMinQuery(final String searchTerm) throws CatalogException { return catalog.getMinQuery(searchTerm); } - /** - * Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. + /** Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. * * @param relationKey the relation to be downloaded. * @param writer the {@link TupleWriter} which will serialize the tuples. * @param dataSink the {@link DataSink} for the tuple destination * @return the query future from which the query status can be looked up. - * @throws DbException if there is an error in the system. - */ + * @throws DbException if there is an error in the system. */ public ListenableFuture startDataStream( final RelationKey relationKey, final TupleWriter writer, final DataSink dataSink) throws DbException { @@ -1510,8 +1390,7 @@ public ListenableFuture startDataStream( } /* Construct the master plan. */ - final CollectConsumer consumer = - new CollectConsumer(schema, operatorId, ImmutableSet.copyOf(scanWorkers)); + final Consumer consumer = new Consumer(schema, operatorId, ImmutableSet.copyOf(scanWorkers)); TupleSink output = new TupleSink(consumer, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); @@ -1524,15 +1403,13 @@ public ListenableFuture startDataStream( } } - /** - * Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. + /** Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. * * @param numTB the number of {@link TupleBatch}es to download from each worker. * @param writer the {@link TupleWriter} which will serialize the tuples. * @param dataSink the {@link DataSink} for the tuple destination * @return the query future from which the query status can be looked up. - * @throws DbException if there is an error in the system. - */ + * @throws DbException if there is an error in the system. */ public ListenableFuture startTestDataStream( final int numTB, final TupleWriter writer, final DataSink dataSink) throws DbException { @@ -1565,8 +1442,7 @@ public ListenableFuture startTestDataStream( } /* Construct the master plan. */ - final CollectConsumer consumer = - new CollectConsumer(schema, operatorId, ImmutableSet.copyOf(scanWorkers)); + final Consumer consumer = new Consumer(schema, operatorId, ImmutableSet.copyOf(scanWorkers)); TupleSink output = new TupleSink(consumer, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); @@ -1579,14 +1455,12 @@ public ListenableFuture startTestDataStream( } } - /** - * @param subqueryId the subquery id. + /** @param subqueryId the subquery id. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public ListenableFuture startSentLogDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -1639,9 +1513,8 @@ public ListenableFuture startSentLogDataStream( workerPlans.put(worker, workerPlan); } - final CollectConsumer consumer = - new CollectConsumer( - addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); final MultiGroupByAggregate aggregate = new MultiGroupByAggregate( @@ -1676,12 +1549,10 @@ public ListenableFuture startSentLogDataStream( } } - /** - * Extracts the set of workers used in a saved, encoded physical plan. + /** Extracts the set of workers used in a saved, encoded physical plan. * * @param plan a {@link List}, cached during execution. - * @return the set of workers used during the execution of this subquery. - */ + * @return the set of workers used during the execution of this subquery. */ @Nonnull private Set getWorkersFromSubqueryPlan(final String plan) { /* @@ -1720,13 +1591,11 @@ private Set getWorkersFromSubqueryPlan(final String plan) { return actualWorkers; } - /** - * Returns the set of workers that executed a particular subquery. + /** Returns the set of workers that executed a particular subquery. * * @param subQueryId the subquery. * @return the set of workers that executed a particular subquery. - * @throws DbException if there is an error in the catalog. - */ + * @throws DbException if there is an error in the catalog. */ private Set getWorkersForSubQuery(final SubQueryId subQueryId) throws DbException { String serializedPlan; try { @@ -1739,13 +1608,11 @@ private Set getWorkersForSubQuery(final SubQueryId subQueryId) throws D return getWorkersFromSubqueryPlan(serializedPlan); } - /** - * @param subqueryId the subquery id. + /** @param subqueryId the subquery id. * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public ListenableFuture startAggregatedSentLogDataStream( final SubQueryId subqueryId, final TupleWriter writer, final DataSink dataSink) throws DbException { @@ -1784,8 +1651,8 @@ public ListenableFuture startAggregatedSentLogDataStream( workerPlans.put(worker, workerPlan); } - final CollectConsumer consumer = - new CollectConsumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); final SingleGroupByAggregate aggregate = new SingleGroupByAggregate( @@ -1823,8 +1690,7 @@ public ListenableFuture startAggregatedSentLogDataStream( } } - /** - * @param subqueryId the desired subquery. + /** @param subqueryId the desired subquery. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param start the earliest time where we need data * @param end the latest time @@ -1833,9 +1699,7 @@ public ListenableFuture startAggregatedSentLogDataStream( * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public QueryFuture startLogDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -1925,9 +1789,8 @@ public QueryFuture startLogDataStream( workerPlans.put(worker, workerPlan); } - final CollectConsumer consumer = - new CollectConsumer( - addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); TupleSink output = new TupleSink(consumer, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); @@ -1956,8 +1819,7 @@ public QueryFuture startLogDataStream( /** Upper bound on the number of bins a profiler can ask for. */ private static final long MAX_BINS = 10000; - /** - * @param subqueryId subquery id. + /** @param subqueryId subquery id. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param start start of the histogram * @param end the end of the histogram @@ -1966,9 +1828,7 @@ public QueryFuture startLogDataStream( * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public QueryFuture startHistogramDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -2041,8 +1901,8 @@ public QueryFuture startHistogramDataStream( } /* Aggregate histogram on master */ - final CollectConsumer consumer = - new CollectConsumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); // sum up the number of workers working final MultiGroupByAggregate sumAggregate = @@ -2079,14 +1939,12 @@ public QueryFuture startHistogramDataStream( } } - /** - * @param subqueryId the subquery id. + /** @param subqueryId the subquery id. * @param fragmentId the fragment id * @param writer writer to get data * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public QueryFuture startRangeDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -2122,8 +1980,8 @@ public QueryFuture startRangeDataStream( } /* Construct the master plan. */ - final CollectConsumer consumer = - new CollectConsumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); // Aggregate range on master final Aggregate sumAggregate = @@ -2153,15 +2011,12 @@ public QueryFuture startRangeDataStream( } } - /** - * @param subqueryId subquery id. + /** @param subqueryId subquery id. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return contributions for operator. - * - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public QueryFuture startContributionsStream( final SubQueryId subqueryId, final long fragmentId, @@ -2202,8 +2057,8 @@ public QueryFuture startContributionsStream( } /* Aggregate on master */ - final CollectConsumer consumer = - new CollectConsumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); // sum up contributions final SingleGroupByAggregate sumAggregate = @@ -2237,13 +2092,11 @@ public QueryFuture startContributionsStream( } } - /** - * Update the {@link MasterCatalog} so that the specified relation has the specified tuple count. + /** Update the {@link MasterCatalog} so that the specified relation has the specified tuple count. * * @param relation the relation to update * @param count the number of tuples in that relation - * @throws DbException if there is an error in the catalog - */ + * @throws DbException if there is an error in the catalog */ public void updateRelationTupleCount(final RelationKey relation, final long count) throws DbException { try { @@ -2253,13 +2106,11 @@ public void updateRelationTupleCount(final RelationKey relation, final long coun } } - /** - * Set the global variable owned by the specified query and named by the specified key to the specified value. + /** Set the global variable owned by the specified query and named by the specified key to the specified value. * * @param queryId the query to whom the variable belongs. * @param key the name of the variable - * @param value the new value for the variable - */ + * @param value the new value for the variable */ public void setQueryGlobal( final long queryId, @Nonnull final String key, @Nonnull final Object value) { Preconditions.checkNotNull(key, "key"); @@ -2267,35 +2118,29 @@ public void setQueryGlobal( queryManager.getQuery(queryId).setGlobal(key, value); } - /** - * Get the value of global variable owned by the specified query and named by the specified key. + /** Get the value of global variable owned by the specified query and named by the specified key. * * @param queryId the query to whom the variable belongs. * @param key the name of the variable - * @return the value of the variable - */ + * @return the value of the variable */ @Nullable public Object getQueryGlobal(final long queryId, @Nonnull final String key) { Preconditions.checkNotNull(key, "key"); return queryManager.getQuery(queryId).getGlobal(key); } - /** - * Return the schema of the specified temp relation in the specified query. + /** Return the schema of the specified temp relation in the specified query. * * @param queryId the query that owns the temp relation * @param name the name of the temporary relation - * @return the schema of the specified temp relation in the specified query - */ + * @return the schema of the specified temp relation in the specified query */ public Schema getTempSchema(@Nonnull final Long queryId, @Nonnull final String name) { return queryManager.getQuery(queryId).getTempSchema(RelationKey.ofTemp(queryId, name)); } - /** - * @param queryId the query id to fetch + /** @param queryId the query id to fetch * @param writerOutput the output stream to write results to. - * @throws DbException if there is an error in the database. - */ + * @throws DbException if there is an error in the database. */ public void getResourceUsage(final long queryId, final DataSink dataSink) throws DbException { Schema schema = Schema.appendColumn(MyriaConstants.RESOURCE_PROFILING_SCHEMA, Type.INT_TYPE, "workerId"); @@ -2315,12 +2160,10 @@ public void getResourceUsage(final long queryId, final DataSink dataSink) throws } } - /** - * @param queryId query id. + /** @param queryId query id. * @param writer writer to get data. * @return resource logs for the query. - * @throws DbException if there is an error when accessing profiling logs. - */ + * @throws DbException if there is an error when accessing profiling logs. */ public ListenableFuture getResourceLog( final long queryId, final TupleWriter writer, final DataSink dataSink) throws DbException { SubQueryId sqId = new SubQueryId(queryId, 0); @@ -2360,9 +2203,8 @@ public ListenableFuture getResourceLog( for (Integer worker : actualWorkers) { workerPlans.put(worker, workerPlan); } - final CollectConsumer consumer = - new CollectConsumer( - addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); TupleSink output = new TupleSink(consumer, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); @@ -2376,13 +2218,11 @@ public ListenableFuture getResourceLog( } } - /** - * Record the fact that this subquery executed this in the catalog. + /** Record the fact that this subquery executed this in the catalog. * * @param subQueryId the id of the subquery. * @param encodedPlan the plan. - * @throws DbException if there is an error in the catalog. - */ + * @throws DbException if there is an error in the catalog. */ public void setQueryPlan(final SubQueryId subQueryId, @Nonnull final String encodedPlan) throws DbException { try { @@ -2392,11 +2232,9 @@ public void setQueryPlan(final SubQueryId subQueryId, @Nonnull final String enco } } - /** - * @param subQueryId the query whose plan to look up. + /** @param subQueryId the query whose plan to look up. * @return the execution plan for this query. - * @throws DbException if there is an error getting the query status. - */ + * @throws DbException if there is an error getting the query status. */ @Nullable public String getQueryPlan(@Nonnull final SubQueryId subQueryId) throws DbException { try { @@ -2406,9 +2244,7 @@ public String getQueryPlan(@Nonnull final SubQueryId subQueryId) throws DbExcept } } - /** - * @return the master catalog. - */ + /** @return the master catalog. */ public MasterCatalog getCatalog() { return catalog; } diff --git a/src/edu/washington/escience/myria/storage/TupleBatch.java b/src/edu/washington/escience/myria/storage/TupleBatch.java index 5ec6db74f..12a225c41 100644 --- a/src/edu/washington/escience/myria/storage/TupleBatch.java +++ b/src/edu/washington/escience/myria/storage/TupleBatch.java @@ -15,15 +15,12 @@ import edu.washington.escience.myria.Type; import edu.washington.escience.myria.column.Column; import edu.washington.escience.myria.column.PrefixColumn; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.PartitionFunction; import edu.washington.escience.myria.proto.TransportProto.TransportMessage; import edu.washington.escience.myria.util.IPCUtils; import net.jcip.annotations.ThreadSafe; -/** - * Container class for a batch of tuples. The goal is to amortize memory management overhead. - * - */ +/** Container class for a batch of tuples. The goal is to amortize memory management overhead. */ @ThreadSafe public class TupleBatch implements ReadableTable, Serializable { /** Required for Java serialization. */ @@ -39,12 +36,10 @@ public class TupleBatch implements ReadableTable, Serializable { /** Whether this TB is an EOI TB. */ private final boolean isEOI; - /** - * EOI TB constructor. + /** EOI TB constructor. * * @param schema schema of the tuples in this batch. - * @param isEoi whether this TupleBatch is an EOI TupleBatch. - */ + * @param isEoi whether this TupleBatch is an EOI TupleBatch. */ private TupleBatch(final Schema schema, final boolean isEoi) { this.schema = schema; numTuples = 0; @@ -56,46 +51,38 @@ private TupleBatch(final Schema schema, final boolean isEoi) { isEOI = isEoi; } - /** - * @param columnNames the new column names. - * @return a shallow copy of the specified TupleBatch with the new column names. - */ + /** @param columnNames the new column names. + * @return a shallow copy of the specified TupleBatch with the new column names. */ public TupleBatch rename(final List columnNames) { Schema newSchema = Schema.of(schema.getColumnTypes(), Objects.requireNonNull(columnNames, "columnNames")); return new TupleBatch(newSchema, columns, numTuples, isEOI); } - /** - * Standard immutable TupleBatch constructor. All fields must be populated before creation and cannot be changed. + /** Standard immutable TupleBatch constructor. All fields must be populated before creation and cannot be changed. * * @param schema schema of the tuples in this batch. Must match columns. * @param columns contains the column-stored data. Must match schema. - * @param numTuples the number of tuples in this TupleBatch. - */ + * @param numTuples the number of tuples in this TupleBatch. */ public TupleBatch( final Schema schema, final List> columns, final int numTuples) { this(schema, columns, numTuples, false); } - /** - * Constructor that gets the number of tuples from the columns. + /** Constructor that gets the number of tuples from the columns. * * @param schema schema of the tuples in this batch. Must match columns. - * @param columns contains the column-stored data. Must match schema. - */ + * @param columns contains the column-stored data. Must match schema. */ public TupleBatch(final Schema schema, final List> columns) { this(schema, columns, columns.get(0).size()); } - /** - * Construct a TupleBatch from the specified components. + /** Construct a TupleBatch from the specified components. * * @param schema schema of the tuples in this batch. Must match columns. * @param columns schema of the tuples in this batch. Must match columns. * @param numTuples the number of tuples in this batch. Must match columns. - * @param isEOI whether this is an EOI TupleBatch. - */ + * @param isEOI whether this is an EOI TupleBatch. */ public TupleBatch( final Schema schema, final List> columns, @@ -119,11 +106,9 @@ public TupleBatch( this.isEOI = isEOI; } - /** - * put the tuple batch into TBB by smashing it into cells and putting them one by one. + /** put the tuple batch into TBB by smashing it into cells and putting them one by one. * - * @param tbb the TBB buffer. - */ + * @param tbb the TBB buffer. */ public final void compactInto(final TupleBatchBuffer tbb) { if (isEOI()) { /* an EOI TB has no data */ @@ -135,14 +120,12 @@ public final void compactInto(final TupleBatchBuffer tbb) { } } - /** - * Return a new TupleBatch that contains only the filtered rows of the current dataset. Note that if some of the + /** Return a new TupleBatch that contains only the filtered rows of the current dataset. Note that if some of the * tuples in this batch are invalid, we will have to map the indices in the specified filter to the "real" indices in * the tuple. * * @param filter the rows to be retained. - * @return a TupleBatch that contains only the filtered rows of the current dataset. - */ + * @return a TupleBatch that contains only the filtered rows of the current dataset. */ public final TupleBatch filter(final BitSet filter) { Preconditions.checkArgument( filter.length() <= numTuples(), @@ -163,12 +146,10 @@ public final TupleBatch filter(final BitSet filter) { return new TupleBatch(schema, newColumns.build(), newNumTuples, isEOI); } - /** - * Return a new TupleBatch that contains only first prefix rows of this batch. + /** Return a new TupleBatch that contains only first prefix rows of this batch. * * @param prefix the number of rows in the prefix to be retained. - * @return a TupleBatch that contains only the filtered rows of the current dataset. - */ + * @return a TupleBatch that contains only the filtered rows of the current dataset. */ @SuppressWarnings({"rawtypes", "unchecked"}) public final TupleBatch prefix(final int prefix) { Preconditions.checkArgument( @@ -239,48 +220,25 @@ public final int numTuples() { return numTuples; } - /** - * Partition this TB using the partition function. The method is implemented by shallow copy of TupleBatches. + /** Partition this TB using the partition function. The method is implemented by shallow copy of TupleBatches. * * @return an array of TBs. The length of the array is the same as the number of partitions. If no tuple presents in a * partition, say the i'th partition, the i'th element in the result array is null. - * @param pf the partition function. - */ + * @param pf the partition function. */ public final TupleBatch[] partition(final PartitionFunction pf) { - TupleBatch[] result = new TupleBatch[pf.numPartition()]; if (isEOI) { + TupleBatch[] result = new TupleBatch[pf.numPartition()]; Arrays.fill(result, this); return result; } - - final int[] partitions = pf.partition(this); - - BitSet[] resultBitSet = new BitSet[result.length]; - for (int i = 0; i < partitions.length; i++) { - int p = partitions[i]; - Preconditions.checkElementIndex(p, result.length); - if (resultBitSet[p] == null) { - resultBitSet[p] = new BitSet(result.length); - } - resultBitSet[p].set(i); - } - - for (int i = 0; i < result.length; i++) { - if (resultBitSet[i] != null) { - result[i] = filter(resultBitSet[i]); - } - } - return result; + return pf.partition(this); } - /** - * Creates a new TupleBatch with only the indicated columns. - * - * Internal implementation of a (non-duplicate-eliminating) PROJECT statement. + /** Creates a new TupleBatch with only the indicated columns. Internal implementation of a (non-duplicate-eliminating) + * PROJECT statement. * * @param remainingColumns zero-indexed array of columns to retain. - * @return a projected TupleBatch. - */ + * @return a projected TupleBatch. */ public final TupleBatch selectColumns(final int[] remainingColumns) { Objects.requireNonNull(remainingColumns); final ImmutableList.Builder> newColumns = new ImmutableList.Builder>(); @@ -291,10 +249,8 @@ public final TupleBatch selectColumns(final int[] remainingColumns) { getSchema().getSubSchema(remainingColumns), newColumns.build(), numTuples, isEOI); } - /** - * @param rows a BitSet flagging the rows to be removed. - * @return a new TB with the specified rows removed. - */ + /** @param rows a BitSet flagging the rows to be removed. + * @return a new TB with the specified rows removed. */ public final TupleBatch filterOut(final BitSet rows) { BitSet inverted = (BitSet) rows.clone(); inverted.flip(0, numTuples); @@ -319,45 +275,35 @@ public final String toString() { return sb.toString(); } - /** - * @return the data columns. - */ + /** @return the data columns. */ public final ImmutableList> getDataColumns() { return columns; } - /** - * @return a TransportMessage encoding the TupleBatch. - */ + /** @return a TransportMessage encoding the TupleBatch. */ public final TransportMessage toTransportMessage() { return IPCUtils.normalDataMessage(columns, numTuples); } - /** - * Create an EOI TupleBatch. + /** Create an EOI TupleBatch. * * @param schema schema. - * @return EOI TB for the schema. - */ + * @return EOI TB for the schema. */ public static final TupleBatch eoiTupleBatch(final Schema schema) { return new TupleBatch(schema, true); } - /** - * @return if the TupleBatch is an EOI. - */ + /** @return if the TupleBatch is an EOI. */ public final boolean isEOI() { return isEOI; } - /** - * Construct a new TupleBatch that equals the current batch with the specified column appended. The number of valid + /** Construct a new TupleBatch that equals the current batch with the specified column appended. The number of valid * tuples in this batch must be the same as the size of the other batch. If this batch is not dense, then * * @param columnName the name of the column to be added. * @param column the column to be added. - * @return a new TupleBatch containing the tuples of this column plus the tuples of the other. - */ + * @return a new TupleBatch containing the tuples of this column plus the tuples of the other. */ public TupleBatch appendColumn(final String columnName, final Column column) { Preconditions.checkArgument( numTuples() == column.size(), diff --git a/src/edu/washington/escience/myria/util/HashUtils.java b/src/edu/washington/escience/myria/util/HashUtils.java index c546cf4ee..8d4e87b67 100644 --- a/src/edu/washington/escience/myria/util/HashUtils.java +++ b/src/edu/washington/escience/myria/util/HashUtils.java @@ -65,9 +65,7 @@ public static int hashRow(final ReadableTable table, final int row) { * @return the hash code of the specified value */ public static int hashValue(final ReadableTable table, final int column, final int row) { - Hasher hasher = HASH_FUNCTIONS[0].newHasher(); - addValue(hasher, table, column, row); - return hasher.hash().asInt(); + return hashValue(table, column, row, 0); } /** @@ -96,9 +94,23 @@ public static int hashValue( * @return the hash code of the specified columns in the specified row of the given table */ public static int hashSubRow(final ReadableTable table, final int[] hashColumns, final int row) { + return hashSubRow(table, hashColumns, row, 0); + } + + /** + * Compute the hash code of the specified columns in the specified row of the given table. + * + * @param table the table containing the values to be hashed + * @param hashColumns the columns to be hashed. Order matters + * @param row the row containing the values to be hashed + * @param seedIndex the index of the chosen hashcode + * @return the hash code of the specified columns in the specified row of the given table + */ + public static int hashSubRow( + final ReadableTable table, final int[] hashColumns, final int row, final int seedIndex) { Objects.requireNonNull(table, "table"); Objects.requireNonNull(hashColumns, "hashColumns"); - Hasher hasher = HASH_FUNCTIONS[0].newHasher(); + Hasher hasher = HASH_FUNCTIONS[seedIndex].newHasher(); for (int column : hashColumns) { addValue(hasher, table, column, row); } diff --git a/src/edu/washington/escience/myria/util/MyriaArrayUtils.java b/src/edu/washington/escience/myria/util/MyriaArrayUtils.java index 469ed5071..c315ad856 100644 --- a/src/edu/washington/escience/myria/util/MyriaArrayUtils.java +++ b/src/edu/washington/escience/myria/util/MyriaArrayUtils.java @@ -1,6 +1,8 @@ package edu.washington.escience.myria.util; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Set; import org.slf4j.LoggerFactory; @@ -10,9 +12,10 @@ import com.google.common.collect.Sets; import com.google.common.primitives.Ints; +import jersey.repackaged.com.google.common.collect.Lists; + /** * Array related utility functions. - * */ public final class MyriaArrayUtils extends org.apache.commons.lang3.ArrayUtils { @@ -30,7 +33,7 @@ private MyriaArrayUtils() {} * @return the filled array. * @param arr the array to fill * @param e the element to fill. - * */ + */ public static Object[] arrayFillAndReturn(final Object[] arr, final Object e) { Arrays.fill(arr, e); return arr; @@ -42,68 +45,33 @@ public static Object[] arrayFillAndReturn(final Object[] arr, final Object e) { * @return the filled array. * @param arr the array to fill * @param e the element to fill. - * */ + */ public static int[] arrayFillAndReturn(final int[] arr, final int e) { Arrays.fill(arr, e); return arr; } /** - * Flatten a 2D array into a 1D array. - * - * @param arr input 2D array - * @return the flattened 1D array () - */ - public static int[] arrayFlatten(final int[][] arr) { - int size = 0; - for (int[] e : arr) { - size += e.length; - } - - int[] result = new int[size]; - int i = 0; - for (int[] e : arr) { - for (int v : e) { - result[i] = v; - i++; - } - } - - return result; - } - - /** - * Flatten a 2D array into a 1D array then sort it. - * - * @param arr input 2D array - * @return the flattened and sorted array - */ - public static int[] arrayFlattenThenSort(final int[][] arr) { - int[] result = arrayFlatten(arr); - Arrays.sort(result); - return result; - } - - /** - * @param length size of 2d index - * @return a 2D index like { {0},{1},{2},..., {n} } + * @param n size of 2d index + * @return a 2D index like { {0},{1},{2},..., {n-1} } */ - public static int[][] create2DVerticalIndex(final int length) { - int[][] result = new int[length][]; - for (int i = 0; i < length; i++) { - result[i] = new int[] {i}; + public static List> create2DVerticalIndexList(final int n) { + List> result = new ArrayList>(); + for (int i = 0; i < n; i++) { + result.add(Lists.newArrayList(i)); } return result; } /** - * @param length size of 2d index - * @return a 2D index like { {0,1,2,...,n} } + * @param n size of 2d index + * @return a 2D index like { {0,1,2,...,n-1} } */ - public static int[][] create2DHorizontalIndex(final int length) { - int[][] result = new int[1][length]; - for (int i = 0; i < length; i++) { - result[0][i] = i; + public static List> create2DHorizontalIndexList(final int n) { + List> result = new ArrayList>(); + result.add(new ArrayList()); + for (int i = 0; i < n; i++) { + result.get(0).add(i); } return result; } @@ -113,7 +81,7 @@ public static int[][] create2DHorizontalIndex(final int length) { * * @param arr input 1D array. * @return 2D array returned. - * */ + */ public static int[][] get2DArray(final int[] arr) { int[][] result = new int[arr.length][]; for (int i = 0; i < arr.length; i++) { @@ -127,7 +95,7 @@ public static int[][] get2DArray(final int[] arr) { * @return An ImmutableSet of the data array if the data array is actually a set * @throws IllegalArgumentException if the data array is not a set. * @param array element type. - * */ + */ public static ImmutableSet checkSet(final E[] maybeSetArray) { ImmutableSet.Builder builder = new ImmutableSet.Builder<>(); for (E i : maybeSetArray) { @@ -146,7 +114,7 @@ public static ImmutableSet checkSet(final E[] maybeSetArray) { * * @param maybeSetArray data array * @return the array itself - * */ + */ public static int[] warnIfNotSet(final int[] maybeSetArray) { try { return checkSet(maybeSetArray); @@ -162,7 +130,7 @@ public static int[] warnIfNotSet(final int[] maybeSetArray) { * @param maybeSetArray data array * @return the array it self * @throws IllegalArgumentException if the data array is not a set. - * */ + */ public static int[] checkSet(final int[] maybeSetArray) { Set tmp = Sets.newHashSet(Ints.asList(maybeSetArray)); if (maybeSetArray.length != tmp.size()) { @@ -179,7 +147,7 @@ public static int[] checkSet(final int[] maybeSetArray) { * @param size the size of another array, i.e. the data array * @return the arrayOfIndices it self * @throws IllegalArgumentException if check fails. - * */ + */ public static int[] checkPositionIndices(final int[] arrayOfIndices, final int size) { for (int i : arrayOfIndices) { Preconditions.checkPositionIndex(i, size); diff --git a/test/edu/washington/escience/myria/api/SerializationTests.java b/test/edu/washington/escience/myria/api/SerializationTests.java index adc370aca..29f227fd4 100644 --- a/test/edu/washington/escience/myria/api/SerializationTests.java +++ b/test/edu/washington/escience/myria/api/SerializationTests.java @@ -9,11 +9,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectReader; -import edu.washington.escience.myria.operator.network.partition.MultiFieldHashPartitionFunction; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; -import edu.washington.escience.myria.operator.network.partition.RoundRobinPartitionFunction; -import edu.washington.escience.myria.operator.network.partition.SingleFieldHashPartitionFunction; -import edu.washington.escience.myria.operator.network.partition.WholeTupleHashPartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.HashDistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.RoundRobinDistributeFunction; public class SerializationTests { @@ -25,60 +23,36 @@ public static void setUp() { } @Test - public void testPartitionFunction() throws Exception { + public void testDistributeFunction() throws Exception { /* Setup */ - ObjectReader reader = mapper.reader(PartitionFunction.class); - PartitionFunction pf; + ObjectReader reader = mapper.reader(DistributeFunction.class); String serialized; - PartitionFunction deserialized; - - /* Single field hash */ - pf = new SingleFieldHashPartitionFunction(5, 3); - serialized = mapper.writeValueAsString(pf); - deserialized = reader.readValue(serialized); - assertEquals(pf.getClass(), deserialized.getClass()); - assertEquals(5, deserialized.numPartition()); - SingleFieldHashPartitionFunction pfSFH = (SingleFieldHashPartitionFunction) deserialized; - assertEquals(3, pfSFH.getIndex()); + DistributeFunction deserialized; /* Multi-field hash */ int multiFieldIndex[] = new int[] {3, 4, 2}; - pf = new MultiFieldHashPartitionFunction(5, multiFieldIndex); - serialized = mapper.writeValueAsString(pf); + DistributeFunction df = new HashDistributeFunction(multiFieldIndex); + serialized = mapper.writeValueAsString(df); deserialized = reader.readValue(serialized); - assertEquals(pf.getClass(), deserialized.getClass()); - assertEquals(5, deserialized.numPartition()); - MultiFieldHashPartitionFunction pfMFH = (MultiFieldHashPartitionFunction) deserialized; + assertEquals(df.getClass(), deserialized.getClass()); + HashDistributeFunction pfMFH = (HashDistributeFunction) deserialized; assertArrayEquals(multiFieldIndex, pfMFH.getIndexes()); - /* Whole tuple hash */ - pf = new WholeTupleHashPartitionFunction(5); - serialized = mapper.writeValueAsString(pf); - deserialized = reader.readValue(serialized); - assertEquals(pf.getClass(), deserialized.getClass()); - assertEquals(5, deserialized.numPartition()); - /* RoundRobin */ - pf = new RoundRobinPartitionFunction(5); - serialized = mapper.writeValueAsString(pf); + df = new RoundRobinDistributeFunction(); + serialized = mapper.writeValueAsString(df); deserialized = reader.readValue(serialized); - assertEquals(pf.getClass(), deserialized.getClass()); - assertEquals(5, deserialized.numPartition()); + assertEquals(df.getClass(), deserialized.getClass()); } @Test - public void testPartitionFunctionWithNullNumPartitions() throws Exception { + public void testDistributeFunctionWithNullNumPartitions() throws Exception { /* Setup */ - ObjectReader reader = mapper.reader(PartitionFunction.class); - PartitionFunction pf; - String serialized; - PartitionFunction deserialized; - - /* Single field hash, as one representative */ - pf = new SingleFieldHashPartitionFunction(null, 3); - serialized = mapper.writeValueAsString(pf); - deserialized = reader.readValue(serialized); - assertEquals(pf.getClass(), deserialized.getClass()); - assertEquals(3, ((SingleFieldHashPartitionFunction) deserialized).getIndex()); + ObjectReader reader = mapper.reader(DistributeFunction.class); + HashDistributeFunction df = new HashDistributeFunction(new int[] {3}); + String serialized = mapper.writeValueAsString(df); + DistributeFunction deserialized = reader.readValue(serialized); + assertEquals(df.getClass(), deserialized.getClass()); + assertEquals(3, ((HashDistributeFunction) deserialized).getIndexes()[0]); } } diff --git a/test/edu/washington/escience/myria/hash/MultiFieldHashPartitionFunctionTest.java b/test/edu/washington/escience/myria/hash/MultiFieldHashPartitionFunctionTest.java index ac88b5414..593fd70d7 100644 --- a/test/edu/washington/escience/myria/hash/MultiFieldHashPartitionFunctionTest.java +++ b/test/edu/washington/escience/myria/hash/MultiFieldHashPartitionFunctionTest.java @@ -2,6 +2,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import java.util.Random; @@ -14,13 +15,12 @@ import edu.washington.escience.myria.Schema; import edu.washington.escience.myria.Type; import edu.washington.escience.myria.operator.BatchTupleSource; -import edu.washington.escience.myria.operator.network.partition.MultiFieldHashPartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.HashPartitionFunction; import edu.washington.escience.myria.storage.TupleBatch; import edu.washington.escience.myria.storage.TupleBatchBuffer; public class MultiFieldHashPartitionFunctionTest { - private static final int NUM_PARTITIONS = 3; private Random rand; // for randomizing numbers @Before @@ -30,30 +30,31 @@ public void setUp() throws Exception { @Test public void testMultiFieldPartitionFunction() { - MultiFieldHashPartitionFunction multiFieldPartitionFunction = - new MultiFieldHashPartitionFunction(NUM_PARTITIONS, new int[] {0, 1}); + HashPartitionFunction multiFieldPartitionFunction = new HashPartitionFunction(new int[] {0, 1}); int numGroups = rand.nextInt(10) + 1; int tuplesPerGroup = rand.nextInt(10) + 1; + multiFieldPartitionFunction.setNumPartitions(numGroups); BatchTupleSource source = generateTupleBatchSource(numGroups, tuplesPerGroup); try { source.open(null); TupleBatch tb = source.nextReady(); assertNotNull(tb); - int[] partitions = multiFieldPartitionFunction.partition(tb); - // for each of the groups, it must map to the same partition - for (int i = 0; i < numGroups; i++) { - int expected = partitions[i * tuplesPerGroup]; - for (int j = 1; j < tuplesPerGroup; j++) { - assertEquals(expected, partitions[i * tuplesPerGroup + j]); - } + TupleBatch[] partitions = multiFieldPartitionFunction.partition(tb); + assertEquals(numGroups, partitions.length); + int s = 0; + for (TupleBatch p : partitions) { + assertTrue(p.numTuples() % tuplesPerGroup == 0); + s += p.numTuples(); } + assertEquals(numGroups * tuplesPerGroup, s); } catch (DbException e) { throw new RuntimeException(e); } } /* - * Generates a tuple batch source with the following schema: a (int), b (int), c (int) + * Generates a tuple batch source with the following schema: a (int), b (int), + * c (int) */ private BatchTupleSource generateTupleBatchSource(int numGroups, int tuplesPerGroup) { final Schema schema = diff --git a/test/edu/washington/escience/myria/util/TestUtils.java b/test/edu/washington/escience/myria/util/TestUtils.java index 84e47fd69..bb72caed3 100644 --- a/test/edu/washington/escience/myria/util/TestUtils.java +++ b/test/edu/washington/escience/myria/util/TestUtils.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import edu.washington.escience.myria.MyriaConstants; @@ -33,9 +34,9 @@ import edu.washington.escience.myria.operator.EmptySink; import edu.washington.escience.myria.operator.Operator; import edu.washington.escience.myria.operator.failures.InitFailureInjector; -import edu.washington.escience.myria.operator.network.GenericShuffleConsumer; +import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; -import edu.washington.escience.myria.operator.network.partition.PartitionFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.parallel.SubQuery; import edu.washington.escience.myria.parallel.SubQueryPlan; @@ -57,26 +58,20 @@ public int compare(final Entry o1, final Entry o2) { private static Random random = null; - /** - * See http://docs.travis-ci.com/user/ci-environment/#Environment-variables + /** See http://docs.travis-ci.com/user/ci-environment/#Environment-variables * - * @return true if the system is currently in a Travis CI build. - */ + * @return true if the system is currently in a Travis CI build. */ public static boolean inTravis() { String travis = System.getenv("TRAVIS"); return (travis != null) && travis.equals("true"); } - /** - * Only run this test in Travis. - */ + /** Only run this test in Travis. */ public static void requireTravis() { Assume.assumeTrue(inTravis()); } - /** - * Skip this test if in Travis. - */ + /** Skip this test if in Travis. */ public static void skipIfInTravis() { Assume.assumeFalse(inTravis()); } @@ -181,9 +176,7 @@ public static HashMap naturalJoin( final int child1JoinColumn, final int child2JoinColumn) { - /** - * join key -> {tuple->num occur} - * */ + /** join key -> {tuple->num occur} */ final HashMap> child1Hash = new HashMap>(); @@ -421,12 +414,10 @@ public static HashMap tupleBatchToTupleBag(final TupleBatchBuffe return result; } - /** - * @param numTuples how many tuples in output + /** @param numTuples how many tuples in output * @param sampleSize how many different values should be created at random (around numTuples/sampleSize duplicates) * @param sorted Generate sorted tuples, sorted by id - * @return - */ + * @return */ public static TupleBatchBuffer generateRandomTuples( final int numTuples, final int sampleSize, final boolean sorted) { final ArrayList> entries = new ArrayList>(); @@ -456,54 +447,50 @@ public static TupleBatchBuffer generateRandomTuples( return tbb; } - /** - * Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the + /** Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the * specified relation key and partition function. * * @param masterSource the source of tuples, from the master. * @param dest the name of the relation into which tuples will be inserted (using overwrite!). - * @param pf how tuples will be partitioned on the cluster. + * @param df * @param workers the set of workers on which the data will be stored. * @return a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the - * specified relation key and partition function. - */ + * specified relation key and partition function. */ public static final SubQuery insertRelation( @Nonnull final Operator masterSource, @Nonnull final RelationKey dest, - @Nonnull final PartitionFunction pf, + @Nonnull final DistributeFunction df, @Nonnull final Set workers) { return insertRelation( masterSource, dest, - pf, + df, ArrayUtils.toPrimitive(workers.toArray(new Integer[workers.size()]))); } - /** - * Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the + /** Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the * specified relation key and partition function. * * @param masterSource the source of tuples, from the master. * @param dest the name of the relation into which tuples will be inserted (using overwrite!). - * @param pf how tuples will be partitioned on the cluster. + * @param df * @param workers the set of workers on which the data will be stored. * @return a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the - * specified relation key and partition function. - */ + * specified relation key and partition function. */ public static final SubQuery insertRelation( @Nonnull final Operator masterSource, @Nonnull final RelationKey dest, - @Nonnull final PartitionFunction pf, + @Nonnull final DistributeFunction df, @Nonnull final int[] workers) { final ExchangePairID id = ExchangePairID.newID(); /* Master plan */ - GenericShuffleProducer sp = new GenericShuffleProducer(masterSource, id, workers, pf); + GenericShuffleProducer sp = + new GenericShuffleProducer(masterSource, new ExchangePairID[] {id}, workers, df); SubQueryPlan masterPlan = new SubQueryPlan(sp); /* Worker plan */ - GenericShuffleConsumer sc = - new GenericShuffleConsumer( - masterSource.getSchema(), id, new int[] {MyriaConstants.MASTER_ID}); + Consumer sc = + new Consumer(masterSource.getSchema(), id, ImmutableSet.of(MyriaConstants.MASTER_ID)); DbInsert insert = new DbInsert(sc, dest, true); Map workerPlans = Maps.newHashMap(); for (int i : workers) { @@ -513,9 +500,7 @@ public static final SubQuery insertRelation( return new SubQuery(masterPlan, workerPlans); } - /** - * Construct a SubQuery that will fail on the master during initialization. Useful for testing failures. - */ + /** Construct a SubQuery that will fail on the master during initialization. Useful for testing failures. */ public static final SubQuery failOnMasterInit() { /* Master plan */ EOSSource src = new EOSSource(); @@ -527,9 +512,7 @@ public static final SubQuery failOnMasterInit() { return new SubQuery(new SubQueryPlan(root), workerPlans); } - /** - * Construct a SubQuery that will fail on one worker during initialization. Useful for testing failures. - */ + /** Construct a SubQuery that will fail on one worker during initialization. Useful for testing failures. */ public static final SubQuery failOnFirstWorkerInit(@Nonnull final int[] workers) { Preconditions.checkElementIndex(1, workers.length); @@ -544,13 +527,11 @@ public static final SubQuery failOnFirstWorkerInit(@Nonnull final int[] workers) return new SubQuery(masterPlan, workerPlans); } - /** - * Returns a {@link TupleBatchBuffer} containing the values 0 to {@code n-1}. The column is of type {@Link - * Type#INT_TYPE} and the column name is {@code "val"}. + /** Returns a {@link TupleBatchBuffer} containing the values 0 to {@code n-1}. The column is of type + * {@Link Type#INT_TYPE} and the column name is {@code "val"}. * * @param n the number of values in the buffer. - * @return a {@link TupleBatchBuffer} containing the values 0 to {@code n-1} - */ + * @return a {@link TupleBatchBuffer} containing the values 0 to {@code n-1} */ public static TupleBatchBuffer range(final int n) { TupleBatchBuffer sourceBuffer = new TupleBatchBuffer(Schema.ofFields(Type.INT_TYPE, "val")); for (int i = 0; i < n; ++i) { diff --git a/testdata/filescan/comma_two_col_int_unix_no_trailing_newline.txt b/testdata/filescan/comma_two_col_int_unix_no_trailing_newline.txt index db02be7ca..2a2b98c98 100644 --- a/testdata/filescan/comma_two_col_int_unix_no_trailing_newline.txt +++ b/testdata/filescan/comma_two_col_int_unix_no_trailing_newline.txt @@ -4,4 +4,4 @@ 7,8 9,10 11,12 -1,2 \ No newline at end of file +1,2 From 4d6a8b3da4b11f6cafb1f229c2bf45e52e643ad5 Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Mon, 19 Dec 2016 13:34:03 +0800 Subject: [PATCH 02/15] change JSON plans accordingly --- .../broadcastJoin_shumo/partition_join.json | 12 +-- .../broadcastJoin_shumo/plan_fragment.py | 6 +- .../come_to_the_party_jwang/party.json | 18 ++--- .../connected_components.json | 18 ++--- .../galaxy_evolution.json | 34 ++++---- jsonQueries/getting_started/global_join.json | 12 +-- jsonQueries/globalJoin_jwang/global_join.json | 12 +-- .../globalJoin_jwang/ingest_smallTable.json | 4 +- jsonQueries/join_for_vis_dominik/join.json | 12 +-- jsonQueries/lca_jwang/lca.json | 30 +++---- .../lsst_vaspol/lsstQuery_oneIteration.json | 18 ++--- jsonQueries/multiIDB_jwang/joinChain.json | 54 ++++++------- jsonQueries/multiIDB_jwang/joinCircle.json | 54 ++++++------- .../multiwayJoin_shumo/plan_fragment.py | 12 +-- .../twoWayJoinSystemTest.json | 12 +-- .../two_dimension_multiway_join.json | 54 +++++-------- .../nullChild_jortiz/ThreeWayLocalJoin.json | 2 +- jsonQueries/radion_queries/SampleScanWR.json | 6 +- jsonQueries/radion_queries/SampleScanWoR.json | 6 +- jsonQueries/radion_queries/SampleWR.json | 6 +- jsonQueries/radion_queries/SampleWoR.json | 6 +- jsonQueries/reachability/reachability.json | 18 ++--- jsonQueries/same_generation_jwang/sg.json | 36 ++++----- jsonQueries/sample_queries/plan_fragment.py | 26 +++--- .../sample_queries/repartition_on_x.json | 6 +- jsonQueries/sample_queries/single_join.json | 12 +-- .../single_join_with_workers.json | 12 +-- .../scalability_valmeida/create_deployment.py | 12 +-- jsonQueries/shortest_paths_jwang/apsp.json | 20 ++--- jsonQueries/shortest_paths_jwang/sssp.json | 18 ++--- jsonQueries/tipsy_leelee/progenitor_all.json | 81 +++++++------------ .../tipsy_leelee/progenitor_only_512.json | 39 +++++---- .../transitive_closure.json | 18 ++--- 33 files changed, 321 insertions(+), 365 deletions(-) diff --git a/jsonQueries/broadcastJoin_shumo/partition_join.json b/jsonQueries/broadcastJoin_shumo/partition_join.json index 5fbe8e893..42b5c747c 100644 --- a/jsonQueries/broadcastJoin_shumo/partition_join.json +++ b/jsonQueries/broadcastJoin_shumo/partition_join.json @@ -13,9 +13,9 @@ }, { "argChild": 1, - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 3, "opType": "ShuffleProducer" @@ -35,9 +35,9 @@ }, { "argChild": 2, - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 4, "opType": "ShuffleProducer" diff --git a/jsonQueries/broadcastJoin_shumo/plan_fragment.py b/jsonQueries/broadcastJoin_shumo/plan_fragment.py index f58e6176f..eef0e63b5 100755 --- a/jsonQueries/broadcastJoin_shumo/plan_fragment.py +++ b/jsonQueries/broadcastJoin_shumo/plan_fragment.py @@ -119,9 +119,9 @@ def scan_and_shuffle(relation_key, relation_name): "argOperatorId": "Shuffle("+relation_name+")", "opId": "Shuffle("+relation_name+")", "opType": "ShuffleProducer", - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" } } fragment = { diff --git a/jsonQueries/come_to_the_party_jwang/party.json b/jsonQueries/come_to_the_party_jwang/party.json index ba7885bc3..8f1eb6cd9 100644 --- a/jsonQueries/come_to_the_party_jwang/party.json +++ b/jsonQueries/come_to_the_party_jwang/party.json @@ -40,9 +40,9 @@ }, { "argChild": 5, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 6, "opType": "ShuffleProducer" @@ -103,9 +103,9 @@ }, { "argChild": 12, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 13, "opType": "ShuffleProducer" @@ -137,9 +137,9 @@ }, { "argChild": 16, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 17, "opType": "ShuffleProducer" diff --git a/jsonQueries/connected_components_jwang/connected_components.json b/jsonQueries/connected_components_jwang/connected_components.json index 73a369da7..6ac73f214 100644 --- a/jsonQueries/connected_components_jwang/connected_components.json +++ b/jsonQueries/connected_components_jwang/connected_components.json @@ -41,9 +41,9 @@ "opId" : 18, "opType" : "ShuffleProducer", "argChild" : 20, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -64,9 +64,9 @@ "opId" : 17, "opType" : "ShuffleProducer", "argChild" : 19, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 1 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [1] } } ] @@ -143,9 +143,9 @@ "opId" : 10, "opType" : "ShuffleProducer", "argChild" : 11, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] diff --git a/jsonQueries/galaxy_evolution_jwang/galaxy_evolution.json b/jsonQueries/galaxy_evolution_jwang/galaxy_evolution.json index f1481d48e..ce900d4bd 100755 --- a/jsonQueries/galaxy_evolution_jwang/galaxy_evolution.json +++ b/jsonQueries/galaxy_evolution_jwang/galaxy_evolution.json @@ -52,9 +52,9 @@ "opName" : "spG", "opType" : "ShuffleProducer", "argChild" : 5, - "argPf" : { - "type" : "MultiFieldHash", - "indexes" : [0, 1] + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0,1] } } ] @@ -94,9 +94,9 @@ "opName" : "spP1", "opType" : "ShuffleProducer", "argChild" : 13, - "argPf" : { - "type" : "MultiFieldHash", - "indexes" : [2, 1] + "distributeFunction" : { + "type" : "Hash", + "indexes" : [2,1] } } ] @@ -155,9 +155,9 @@ "opName" : "spP2", "opType" : "ShuffleProducer", "argChild" : 35, - "argPf" : { - "type" : "MultiFieldHash", - "indexes" : [2, 0] + "distributeFunction" : { + "type" : "Hash", + "indexes" : [2,0] } } ] @@ -209,8 +209,8 @@ "opName" : "spE", "opType" : "ShuffleProducer", "argChild" : 20, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -322,8 +322,8 @@ "opName" : "spJoinG", "opType" : "ShuffleProducer", "argChild" : 32, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -362,8 +362,8 @@ "opName" : "spJoinE1", "opType" : "ShuffleProducer", "argChild" : 34, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,2] } } @@ -402,8 +402,8 @@ "opName" : "spJoinE2", "opType" : "ShuffleProducer", "argChild" : 39, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1,2] } } diff --git a/jsonQueries/getting_started/global_join.json b/jsonQueries/getting_started/global_join.json index d4afb76b2..d43fab628 100644 --- a/jsonQueries/getting_started/global_join.json +++ b/jsonQueries/getting_started/global_join.json @@ -13,9 +13,9 @@ }, { "argChild": 7, - "argPf": { - "index": 0, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 0, "opType": "ShuffleProducer" @@ -35,9 +35,9 @@ }, { "argChild": 1, - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 6, "opType": "ShuffleProducer" diff --git a/jsonQueries/globalJoin_jwang/global_join.json b/jsonQueries/globalJoin_jwang/global_join.json index 51fcdadc2..f3952118a 100644 --- a/jsonQueries/globalJoin_jwang/global_join.json +++ b/jsonQueries/globalJoin_jwang/global_join.json @@ -13,9 +13,9 @@ }, { "argChild": 6, - "argPf": { - "index": 0, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 0, "opType": "ShuffleProducer" @@ -35,9 +35,9 @@ }, { "argChild": 1, - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 6, "opType": "ShuffleProducer" diff --git a/jsonQueries/globalJoin_jwang/ingest_smallTable.json b/jsonQueries/globalJoin_jwang/ingest_smallTable.json index f994f7d1c..5caf8baab 100644 --- a/jsonQueries/globalJoin_jwang/ingest_smallTable.json +++ b/jsonQueries/globalJoin_jwang/ingest_smallTable.json @@ -13,8 +13,8 @@ "bytes" : "MSA0NAoyIDUxCjQ2IDE3CjYzIDM0CjU0IDYzCjIwIDk0CjEyIDY2Cjc5IDQyCjEgMTAKODggMjAKMTAgNDIKNTYgNDQKMTAgMTIKNzkgMzcKMzAgNjYKODMgMTMKMzEgMQozMSA5OQo4MSAzNQo3MCAyNgo0IDUxCjE1IDY2Cjg4IDY2CjI3IDE3CjMxIDgyCjc2IDc0Cjk2IDY1CjYyIDIyCjkwIDU5CjEzIDI5CjQ0IDQyCjM1IDYyCjk5IDE1Cjk1IDc3CjEwIDcwCjI0IDMwCjgyIDY0CjQ0IDQ4CjY1IDc0CjE4IDg1CjQ5IDE0Cjc1IDk5CjU3IDk1CjQyIDk2CjQxIDY5CjE0IDY1CjE2IDExCjcyIDIyCjc2IDgyCjY2IDY4Cjc0IDg4CjQ3IDYKNTYgMAo2IDkKNTAgODAKNiAzMQo3NiA0NAo0OSAzMAo0NyAxNgo4MiA3NwoxIDgxCjIwIDQwCjE4IDU2CjI4IDkyCjU4IDE2CjgyIDEzCjcxIDc1CjYwIDQxCjIzIDkKMiA1MQo4NiA5NQo4IDgxCjk3IDc5CjE4IDQxCjg5IDQ4CjU5IDUxCjIxIDg2CjYzIDc2CjQyIDIyCjczIDM4CjI0IDE3CjggMzQKNzggMTUKOTMgMTUKMzEgMjIKNzMgMjkKOTMgMTYKODcgOTUKNSA1Nwo0MiA4OAoxNSA4NwozOCA5NwowIDc2CjU3IDUxCjMwIDE5CjUyIDI4CjQyIDE0CjczIDI4CjM3IDY5CjQzIDQ3Cg==" }, "partitionFunction": { - "type": "SingleFieldHash", - "index": 0 + "type": "Hash", + "indexes": [0] }, "overwrite": true, "delimiter":" " diff --git a/jsonQueries/join_for_vis_dominik/join.json b/jsonQueries/join_for_vis_dominik/join.json index a5ce79a3c..0daa46aa1 100644 --- a/jsonQueries/join_for_vis_dominik/join.json +++ b/jsonQueries/join_for_vis_dominik/join.json @@ -90,9 +90,9 @@ { "opName": "MyriaShuffleProducer(h($1))", "opId": 4, - "argPf": { - "type": "SingleFieldHash", - "index": 1 + "distributeFunction": { + "type": "Hash", + "indexes": [1] }, "opType": "ShuffleProducer", "argChild": 7 @@ -134,9 +134,9 @@ { "opName": "MyriaShuffleProducer(h($0))", "opId": 2, - "argPf": { - "type": "SingleFieldHash", - "index": 0 + "distributeFunction": { + "type": "Hash", + "indexes": [0] }, "opType": "ShuffleProducer", "argChild": 9 diff --git a/jsonQueries/lca_jwang/lca.json b/jsonQueries/lca_jwang/lca.json index 26669343e..46a4fb18c 100755 --- a/jsonQueries/lca_jwang/lca.json +++ b/jsonQueries/lca_jwang/lca.json @@ -52,9 +52,9 @@ "opName" : "spP", "opType" : "ShuffleProducer", "argChild" : 5, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -94,9 +94,9 @@ "opName" : "spC1", "opType" : "ShuffleProducer", "argChild" : 9, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -147,8 +147,8 @@ "opName" : "spC2", "opType" : "ShuffleProducer", "argChild" : 18, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -194,9 +194,9 @@ "opName" : "spA", "opType" : "ShuffleProducer", "argChild" : 20, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 1 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [1] } } ] @@ -341,8 +341,8 @@ "opName" : "spJoinA", "opType" : "ShuffleProducer", "argChild" : 40, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -514,8 +514,8 @@ "opName" : "spJoinL2", "opType" : "ShuffleProducer", "argChild" : 45, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } diff --git a/jsonQueries/lsst_vaspol/lsstQuery_oneIteration.json b/jsonQueries/lsst_vaspol/lsstQuery_oneIteration.json index 84009be94..beb86a61b 100644 --- a/jsonQueries/lsst_vaspol/lsstQuery_oneIteration.json +++ b/jsonQueries/lsst_vaspol/lsstQuery_oneIteration.json @@ -13,12 +13,9 @@ }, { "argChild": 1, - "argPf": { - "indexes": [ - "0", - "1" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [0,1], + "type": "Hash" }, "opId": 3, "opType": "ShuffleProducer" @@ -43,12 +40,9 @@ }, { "argChild": 2, - "argPf": { - "indexes": [ - "0", - "1" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [0,1], + "type": "Hash" }, "opId": 7, "opType": "ShuffleProducer" diff --git a/jsonQueries/multiIDB_jwang/joinChain.json b/jsonQueries/multiIDB_jwang/joinChain.json index cfbc77ad7..4b5b9c628 100644 --- a/jsonQueries/multiIDB_jwang/joinChain.json +++ b/jsonQueries/multiIDB_jwang/joinChain.json @@ -49,9 +49,9 @@ }, { "argChild": 6, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 7, "opType": "ShuffleProducer" @@ -107,9 +107,9 @@ }, { "argChild": 12, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 13, "opType": "ShuffleProducer" @@ -148,9 +148,9 @@ }, { "argChild": 16, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 17, "opType": "ShuffleProducer" @@ -166,9 +166,9 @@ }, { "argChild": 19, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 20, "opType": "ShuffleProducer" @@ -188,9 +188,9 @@ }, { "argChild": 25, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 26, "opType": "ShuffleProducer" @@ -265,9 +265,9 @@ }, { "argChild": 30, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 47, "opType": "ShuffleProducer" @@ -283,9 +283,9 @@ }, { "argChild": 31, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 32, "opType": "ShuffleProducer" @@ -305,9 +305,9 @@ }, { "argChild": 33, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 34, "opType": "ShuffleProducer" @@ -387,9 +387,9 @@ }, { "argChild": 40, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 41, "opType": "ShuffleProducer" diff --git a/jsonQueries/multiIDB_jwang/joinCircle.json b/jsonQueries/multiIDB_jwang/joinCircle.json index ddda934ad..2f73fbf53 100644 --- a/jsonQueries/multiIDB_jwang/joinCircle.json +++ b/jsonQueries/multiIDB_jwang/joinCircle.json @@ -49,9 +49,9 @@ }, { "argChild": 6, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 7, "opType": "ShuffleProducer" @@ -126,9 +126,9 @@ }, { "argChild": 15, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 16, "opType": "ShuffleProducer" @@ -144,9 +144,9 @@ }, { "argChild": 17, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 18, "opType": "ShuffleProducer" @@ -166,9 +166,9 @@ }, { "argChild": 19, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 20, "opType": "ShuffleProducer" @@ -243,9 +243,9 @@ }, { "argChild": 28, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 29, "opType": "ShuffleProducer" @@ -261,9 +261,9 @@ }, { "argChild": 30, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 31, "opType": "ShuffleProducer" @@ -283,9 +283,9 @@ }, { "argChild": 32, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 33, "opType": "ShuffleProducer" @@ -365,9 +365,9 @@ }, { "argChild": 41, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 42, "opType": "ShuffleProducer" @@ -383,9 +383,9 @@ }, { "argChild": 43, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 44, "opType": "ShuffleProducer" diff --git a/jsonQueries/multiwayJoin_shumo/plan_fragment.py b/jsonQueries/multiwayJoin_shumo/plan_fragment.py index 6f3d3ad2a..4cfa334fb 100755 --- a/jsonQueries/multiwayJoin_shumo/plan_fragment.py +++ b/jsonQueries/multiwayJoin_shumo/plan_fragment.py @@ -151,10 +151,10 @@ def scan_R_then_partition(): "opId" : "Shuffle(R)", "argChild" : "Scan(R)", "argOperatorId" : "hash(followee)", - "argPf" : + "distributeFunction" : { - "type" : "SingleFieldHash", - "index" : 1 + "type" : "Hash", + "indexes" : [1] } } fragment = { @@ -175,10 +175,10 @@ def scan_S_then_partition(): "opId" : "Shuffle(S)", "argChild" : "Scan(S)", "argOperatorId" : "hash(follower)", - "argPf" : + "distributeFunction" : { - "type" : "SingleFieldHash", - "index" : 0 + "type" : "Hash", + "indexes" : [0] } } fragment = { diff --git a/jsonQueries/multiwayJoin_shumo/twoWayJoinSystemTest.json b/jsonQueries/multiwayJoin_shumo/twoWayJoinSystemTest.json index c72ed4032..85950cccc 100644 --- a/jsonQueries/multiwayJoin_shumo/twoWayJoinSystemTest.json +++ b/jsonQueries/multiwayJoin_shumo/twoWayJoinSystemTest.json @@ -13,9 +13,9 @@ }, { "argChild":0, - "argPf":{ - "index":0, - "type":"SingleFieldHash" + "distributeFunction":{ + "indexes":[0], + "type":"Hash" }, "opId":1, "opType":"ShuffleProducer" @@ -35,9 +35,9 @@ }, { "argChild":2, - "argPf":{ - "index":0, - "type":"SingleFieldHash" + "distributeFunction":{ + "indexes":[0], + "type":"Hash" }, "opId":9, "opType":"ShuffleProducer" diff --git a/jsonQueries/multiwayJoin_shumo/two_dimension_multiway_join.json b/jsonQueries/multiwayJoin_shumo/two_dimension_multiway_join.json index cddba9e36..5d293750f 100644 --- a/jsonQueries/multiwayJoin_shumo/two_dimension_multiway_join.json +++ b/jsonQueries/multiwayJoin_shumo/two_dimension_multiway_join.json @@ -13,24 +13,17 @@ }, { "argChild":1, - "hashedColumns": [0], - "cellPartition":[ - [ - 0, - 1 + "distributeFunction": { + "hashedColumns": [0], + "mappedHCDimensions":[0], + "cellPartition":[ + [0,1],[2,3] ], - [ - 2, - 3 - ] - ], - "mappedHCDimensions":[0], - "hyperCubeDimensions":[ - 2, - 2 - ], + "hyperCubeDimensions":[2,2], + "type":"HyperCube" + }, "opId":3, - "opType":"HyperShuffleProducer" + "opType":"HyperCubeShuffleProducer" } ] }, @@ -47,24 +40,17 @@ }, { "argChild":2, - "hashedColumns": [1], - "mappedHCDimensions":[1], - "cellPartition":[ - [ - 0, - 2 + "distributeFunction": { + "hashedColumns": [1], + "mappedHCDimensions":[1], + "cellPartition":[ + [0,2],[1,3] ], - [ - 1, - 3 - ] - ], - "hyperCubeDimensions":[ - 2, - 2 - ], + "hyperCubeDimensions":[2,2], + "type":"HyperCube" + }, "opId":8, - "opType":"HyperShuffleProducer" + "opType":"HyperCubeShuffleProducer" } ] }, @@ -73,12 +59,12 @@ { "argOperatorId":3, "opId":4, - "opType":"HyperShuffleConsumer" + "opType":"HyperCubeShuffleConsumer" }, { "argOperatorId":8, "opId":5, - "opType":"HyperShuffleConsumer" + "opType":"HyperCubeShuffleConsumer" }, { "argChild1":4, diff --git a/jsonQueries/nullChild_jortiz/ThreeWayLocalJoin.json b/jsonQueries/nullChild_jortiz/ThreeWayLocalJoin.json index 2d229ee90..e19dcd376 100644 --- a/jsonQueries/nullChild_jortiz/ThreeWayLocalJoin.json +++ b/jsonQueries/nullChild_jortiz/ThreeWayLocalJoin.json @@ -82,4 +82,4 @@ ], "type": "SubQuery" } -} \ No newline at end of file +} diff --git a/jsonQueries/radion_queries/SampleScanWR.json b/jsonQueries/radion_queries/SampleScanWR.json index 4e449084e..502e51d83 100644 --- a/jsonQueries/radion_queries/SampleScanWR.json +++ b/jsonQueries/radion_queries/SampleScanWR.json @@ -73,9 +73,9 @@ }, { "argChild":7, - "argPf":{ - "index":0, - "type":"IdentityHash" + "distributeFunction":{ + "indexes":[0], + "type":"Identity" }, "opType":"ShuffleProducer", "opId":8, diff --git a/jsonQueries/radion_queries/SampleScanWoR.json b/jsonQueries/radion_queries/SampleScanWoR.json index cfe46f8e6..d610c36df 100644 --- a/jsonQueries/radion_queries/SampleScanWoR.json +++ b/jsonQueries/radion_queries/SampleScanWoR.json @@ -73,9 +73,9 @@ }, { "argChild":7, - "argPf":{ - "index":0, - "type":"IdentityHash" + "distributeFunction":{ + "indexes":[0], + "type":"Identity" }, "opType":"ShuffleProducer", "opId":8, diff --git a/jsonQueries/radion_queries/SampleWR.json b/jsonQueries/radion_queries/SampleWR.json index d4b021049..027dcb896 100644 --- a/jsonQueries/radion_queries/SampleWR.json +++ b/jsonQueries/radion_queries/SampleWR.json @@ -80,9 +80,9 @@ { "argChild":8, "opId":9, - "argPf":{ - "index":0, - "type":"IdentityHash" + "distributeFunction":{ + "indexes":[0], + "type":"Identity" }, "opType":"ShuffleProducer", "opName":"MyriaShuffleProducer($0)" diff --git a/jsonQueries/radion_queries/SampleWoR.json b/jsonQueries/radion_queries/SampleWoR.json index 99eb5ef9c..ad63d9f56 100644 --- a/jsonQueries/radion_queries/SampleWoR.json +++ b/jsonQueries/radion_queries/SampleWoR.json @@ -80,9 +80,9 @@ { "argChild":8, "opId":9, - "argPf":{ - "index":0, - "type":"IdentityHash" + "distributeFunction":{ + "indexes":[0], + "type":"Identity" }, "opType":"ShuffleProducer", "opName":"MyriaShuffleProducer($0)" diff --git a/jsonQueries/reachability/reachability.json b/jsonQueries/reachability/reachability.json index c7debc37a..ed9f927f7 100644 --- a/jsonQueries/reachability/reachability.json +++ b/jsonQueries/reachability/reachability.json @@ -41,9 +41,9 @@ "opId" : 19, "opType" : "ShuffleProducer", "argChild" : 4, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -64,9 +64,9 @@ "opId" : 18, "opType" : "ShuffleProducer", "argChild" : 17, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -140,9 +140,9 @@ "opId" : 12, "opType" : "ShuffleProducer", "argChild" : 10, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] diff --git a/jsonQueries/same_generation_jwang/sg.json b/jsonQueries/same_generation_jwang/sg.json index 5eb69f1a4..307f2c899 100644 --- a/jsonQueries/same_generation_jwang/sg.json +++ b/jsonQueries/same_generation_jwang/sg.json @@ -77,8 +77,8 @@ "opId" : 46, "opType" : "ShuffleProducer", "argChild" : 45, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -118,8 +118,8 @@ "opId" : 6, "opType" : "ShuffleProducer", "argChild" : 5, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -173,9 +173,9 @@ "opId" : 11, "opType" : "ShuffleProducer", "argChild" : 10, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -192,9 +192,9 @@ "opId" : 13, "opType" : "ShuffleProducer", "argChild" : 12, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -211,9 +211,9 @@ "opId" : 23, "opType" : "ShuffleProducer", "argChild" : 22, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -246,9 +246,9 @@ "opId" : 18, "opType" : "ShuffleProducer", "argChild" : 16, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 1 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [1] } } ] @@ -302,8 +302,8 @@ "opId" : 27, "opType" : "ShuffleProducer", "argChild" : 26, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } diff --git a/jsonQueries/sample_queries/plan_fragment.py b/jsonQueries/sample_queries/plan_fragment.py index 160381b85..fe1ffb3cb 100755 --- a/jsonQueries/sample_queries/plan_fragment.py +++ b/jsonQueries/sample_queries/plan_fragment.py @@ -53,9 +53,9 @@ def repartition_on_x(): 'opId' : 'Scatter', 'argChild' : 'Scan', 'argOperatorId' : 'hash(follower)', - 'argPf' : { - 'type' : 'SingleFieldHash', - 'index' : 0 + 'distributeFunction' : { + 'type' : 'Hash', + 'indexes' : [0] } } gather = { @@ -107,9 +107,9 @@ def single_join(): 'opId' : 'Scatter0', 'argChild' : 'Scan0', 'argOperatorId' : 'hash(x)', - 'argPf' : { - 'type' : 'SingleFieldHash', - 'index' : 0 + 'distributeFunction' : { + 'type' : 'Hash', + 'indexes' : [0] } } gather0 = { @@ -136,9 +136,9 @@ def single_join(): 'opId' : 'Scatter1', 'argChild' : 'Scan1', 'argOperatorId' : 'hash(y)', - 'argPf' : { - 'type' : 'SingleFieldHash', - 'index' : 1 + 'distributeFunction' : { + 'type' : 'Hash', + 'indexes' : [1] } } gather1 = { @@ -218,7 +218,7 @@ def ingest_tipsy_rr(): 'opId' : 'Scatter', 'argChild' : 'Scan', 'argOperatorId' : 'RoundRobin', - 'argPf' : { + 'distributeFunction' : { 'type' : 'RoundRobin' } } @@ -268,9 +268,9 @@ def ingest_tipsy_hash_iorder(): 'opId' : 'Scatter', 'argChild' : 'Scan', 'argOperatorId' : 'hash(iorder)', - 'argPf' : { - 'type' : 'SingleFieldHash', - 'index' : 0 + 'distributeFunction' : { + 'type' : 'Hash', + 'indexes' : [0] } } scan_fragment = { diff --git a/jsonQueries/sample_queries/repartition_on_x.json b/jsonQueries/sample_queries/repartition_on_x.json index 934e6075f..555fe1bae 100644 --- a/jsonQueries/sample_queries/repartition_on_x.json +++ b/jsonQueries/sample_queries/repartition_on_x.json @@ -13,9 +13,9 @@ }, { "argChild": 0, - "argPf": { - "index": 0, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 1, "opType": "ShuffleProducer" diff --git a/jsonQueries/sample_queries/single_join.json b/jsonQueries/sample_queries/single_join.json index c2ef2f406..11139faeb 100644 --- a/jsonQueries/sample_queries/single_join.json +++ b/jsonQueries/sample_queries/single_join.json @@ -13,9 +13,9 @@ }, { "argChild": 6, - "argPf": { - "index": 0, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 0, "opType": "ShuffleProducer" @@ -35,9 +35,9 @@ }, { "argChild": 1, - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 7, "opType": "ShuffleProducer" diff --git a/jsonQueries/sample_queries/single_join_with_workers.json b/jsonQueries/sample_queries/single_join_with_workers.json index bb97e13a8..57b2d42cd 100644 --- a/jsonQueries/sample_queries/single_join_with_workers.json +++ b/jsonQueries/sample_queries/single_join_with_workers.json @@ -13,9 +13,9 @@ }, { "argChild": 0, - "argPf": { - "index": 0, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 1, "opType": "ShuffleProducer" @@ -39,9 +39,9 @@ }, { "argChild": 2, - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 3, "opType": "ShuffleProducer" diff --git a/jsonQueries/scalability_valmeida/create_deployment.py b/jsonQueries/scalability_valmeida/create_deployment.py index e3f1a987e..c6e1cb669 100644 --- a/jsonQueries/scalability_valmeida/create_deployment.py +++ b/jsonQueries/scalability_valmeida/create_deployment.py @@ -230,9 +230,9 @@ "argOperatorId": "0", "opId": "SP1", "opType": "ShuffleProducer", - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" } } @@ -253,9 +253,9 @@ "argOperatorId": "1", "opId": "SP2", "opType": "ShuffleProducer", - "argPf": { - "index": 1, - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" } } diff --git a/jsonQueries/shortest_paths_jwang/apsp.json b/jsonQueries/shortest_paths_jwang/apsp.json index cac498153..119eedcfe 100644 --- a/jsonQueries/shortest_paths_jwang/apsp.json +++ b/jsonQueries/shortest_paths_jwang/apsp.json @@ -85,8 +85,8 @@ "opId" : 6, "opType" : "ShuffleProducer", "argChild" : 5, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } @@ -132,9 +132,9 @@ "opId" : 11, "opType" : "ShuffleProducer", "argChild" : 10, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 1 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [1] } } ] @@ -151,9 +151,9 @@ "opId" : 13, "opType" : "ShuffleProducer", "argChild" : 12, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -223,8 +223,8 @@ "opId" : 18, "opType" : "ShuffleProducer", "argChild" : 17, - "argPf" : { - "type" : "MultiFieldHash", + "distributeFunction" : { + "type" : "Hash", "indexes" : [0,1] } } diff --git a/jsonQueries/shortest_paths_jwang/sssp.json b/jsonQueries/shortest_paths_jwang/sssp.json index c3af3c409..c3e240f8b 100644 --- a/jsonQueries/shortest_paths_jwang/sssp.json +++ b/jsonQueries/shortest_paths_jwang/sssp.json @@ -59,9 +59,9 @@ "opId" : 6, "opType" : "ShuffleProducer", "argChild" : 5, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -125,9 +125,9 @@ "opId" : 13, "opType" : "ShuffleProducer", "argChild" : 12, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -190,9 +190,9 @@ "opId" : 18, "opType" : "ShuffleProducer", "argChild" : 17, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] diff --git a/jsonQueries/tipsy_leelee/progenitor_all.json b/jsonQueries/tipsy_leelee/progenitor_all.json index d65588380..562f3d37c 100644 --- a/jsonQueries/tipsy_leelee/progenitor_all.json +++ b/jsonQueries/tipsy_leelee/progenitor_all.json @@ -496,9 +496,9 @@ }, { "argChild": 27, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 28, "opType": "ShuffleProducer" @@ -524,9 +524,9 @@ }, { "argChild": 29, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 30, "opType": "ShuffleProducer" @@ -546,9 +546,9 @@ }, { "argChild": 31, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 32, "opType": "ShuffleProducer" @@ -588,9 +588,9 @@ }, { "argChild": 36, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 37, "opType": "ShuffleProducer" @@ -631,13 +631,9 @@ }, { "argChild": 41, - "argPf": { - "indexes": [ - "1", - "2", - "3" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [1,2,3], + "type": "Hash" }, "opId": 42, "opType": "ShuffleProducer" @@ -681,12 +677,9 @@ }, { "argChild": 46, - "argPf": { - "indexes": [ - "0", - "2" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [0,2], + "type": "Hash" }, "opId": 48, "opType": "ShuffleProducer" @@ -717,13 +710,9 @@ }, { "argChild": 50, - "argPf": { - "indexes": [ - "0", - "1", - "2" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [0,1,2], + "type": "Hash" }, "opId": 51, "opType": "ShuffleProducer" @@ -739,13 +728,9 @@ }, { "argChild": 52, - "argPf": { - "indexes": [ - "0", - "2", - "3" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [0,2,3], + "type": "Hash" }, "opId": 54, "opType": "ShuffleProducer" @@ -805,12 +790,9 @@ }, { "argChild": 58, - "argPf": { - "indexes": [ - "2", - "1" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [2,1], + "type": "Hash" }, "opId": 59, "opType": "ShuffleProducer" @@ -1313,12 +1295,9 @@ }, { "argChild": 87, - "argPf": { - "indexes": [ - "0", - "2" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [0,2], + "type": "Hash" }, "opId": 88, "opType": "ShuffleProducer" diff --git a/jsonQueries/tipsy_leelee/progenitor_only_512.json b/jsonQueries/tipsy_leelee/progenitor_only_512.json index 8dedb3683..7d0131846 100644 --- a/jsonQueries/tipsy_leelee/progenitor_only_512.json +++ b/jsonQueries/tipsy_leelee/progenitor_only_512.json @@ -19,9 +19,9 @@ }, { "argChild": 1, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 2, "opType": "ShuffleProducer" @@ -47,9 +47,9 @@ }, { "argChild": 3, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 4, "opType": "ShuffleProducer" @@ -89,9 +89,9 @@ }, { "argChild": 7, - "argPf": { - "index": "1", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [1], + "type": "Hash" }, "opId": 8, "opType": "ShuffleProducer" @@ -111,9 +111,9 @@ }, { "argChild": 9, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 10, "opType": "ShuffleProducer" @@ -154,12 +154,9 @@ }, { "argChild": 13, - "argPf": { - "indexes": [ - "1", - "2" - ], - "type": "MultiFieldHash" + "distributeFunction": { + "indexes": [1,2], + "type": "Hash" }, "opId": 14, "opType": "ShuffleProducer" @@ -206,9 +203,9 @@ }, { "argChild": 18, - "argPf": { - "index": "0", - "type": "SingleFieldHash" + "distributeFunction": { + "indexes": [0], + "type": "Hash" }, "opId": 19, "opType": "ShuffleProducer" diff --git a/jsonQueries/transitive_closure_jwang/transitive_closure.json b/jsonQueries/transitive_closure_jwang/transitive_closure.json index 25bb6864e..37862007e 100644 --- a/jsonQueries/transitive_closure_jwang/transitive_closure.json +++ b/jsonQueries/transitive_closure_jwang/transitive_closure.json @@ -41,9 +41,9 @@ "opId" : 5, "opType" : "ShuffleProducer", "argChild" : 4, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] @@ -64,9 +64,9 @@ "opId" : 16, "opType" : "ShuffleProducer", "argChild" : 17, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 1 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [1] } } ] @@ -137,9 +137,9 @@ "type" : "DupElim" }, "argChild" : 12, - "argPf" : { - "type" : "SingleFieldHash", - "index" : 0 + "distributeFunction" : { + "type" : "Hash", + "indexes" : [0] } } ] From 673da3c1fd7bed49ca2aafc0a031380f74ffa7ea Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Tue, 27 Dec 2016 11:02:19 -0800 Subject: [PATCH 03/15] fix dataset API for broadcast relations --- .../escience/myria/coordinator/MasterCatalog.java | 12 ++++++------ .../escience/myria/parallel/Server.java | 15 +++++++++++---- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/src/edu/washington/escience/myria/coordinator/MasterCatalog.java b/src/edu/washington/escience/myria/coordinator/MasterCatalog.java index daf548f01..a485ae6ed 100644 --- a/src/edu/washington/escience/myria/coordinator/MasterCatalog.java +++ b/src/edu/washington/escience/myria/coordinator/MasterCatalog.java @@ -1252,7 +1252,7 @@ protected Object job(final SQLiteConnection sqliteConnection) * @param howDistributed how the dataset was distributed. * @throws CatalogException if there is an error in the catalog. */ - public void updateHowPartitioned(final RelationKey key, final HowDistributed howDistributed) + public void updateHowDistributed(final RelationKey key, final HowDistributed howDistributed) throws CatalogException { if (isClosed) { throw new CatalogException("Catalog is closed."); @@ -1328,21 +1328,21 @@ protected DatasetStatus job(final SQLiteConnection sqliteConnection) long numTuples = statement.columnLong(0); long queryId = statement.columnLong(1); String created = statement.columnString(2); - HowDistributed howPartitioned; + HowDistributed howDistributed; try { - howPartitioned = + howDistributed = MyriaJsonMapperProvider.getMapper() .readValue(statement.columnString(3), HowDistributed.class); } catch (final IOException e) { LOGGER.debug( - "Error deserializing howPartitioned for dataset #{}", + "Error deserializing howDistributed for dataset #{}", relationKey.toString(), e); - howPartitioned = new HowDistributed(null, null); + howDistributed = new HowDistributed(null, null); } statement.dispose(); return new DatasetStatus( - relationKey, schema, numTuples, queryId, created, howPartitioned); + relationKey, schema, numTuples, queryId, created, howDistributed); } catch (final SQLiteException e) { throw new CatalogException(e); } diff --git a/src/edu/washington/escience/myria/parallel/Server.java b/src/edu/washington/escience/myria/parallel/Server.java index 717e25b6f..e25bcd7a1 100644 --- a/src/edu/washington/escience/myria/parallel/Server.java +++ b/src/edu/washington/escience/myria/parallel/Server.java @@ -113,6 +113,7 @@ import edu.washington.escience.myria.operator.network.CollectProducer; import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; +import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.operator.network.distribute.HowDistributed; import edu.washington.escience.myria.parallel.ipc.IPCConnectionPool; @@ -838,7 +839,7 @@ public DatasetStatus ingestDataset( } // updating the partition function only after it's successfully ingested. - updateHowPartitioned(relationKey, new HowDistributed(df, workersArray)); + updateHowDistributed(relationKey, new HowDistributed(df, workersArray)); return getDatasetStatus(relationKey); } @@ -932,7 +933,7 @@ public DatasetStatus parallelIngestDataset( throw new DbException("Error executing query", e.getCause()); } - updateHowPartitioned(relationKey, new HowDistributed(distributeFunction, workersArray)); + updateHowDistributed(relationKey, new HowDistributed(distributeFunction, workersArray)); return getDatasetStatus(relationKey); } @@ -1237,10 +1238,10 @@ public Schema getSchema(final RelationKey relationKey) throws CatalogException { /** @param key the relation key. * @param howPartitioned how the dataset was partitioned. * @throws DbException if there is an catalog exception. */ - public void updateHowPartitioned(final RelationKey key, final HowDistributed howPartitioned) + public void updateHowDistributed(final RelationKey key, final HowDistributed howDistributed) throws DbException { try { - catalog.updateHowPartitioned(key, howPartitioned); + catalog.updateHowDistributed(key, howDistributed); } catch (CatalogException e) { throw new DbException(e); } @@ -1378,6 +1379,12 @@ public ListenableFuture startDataStream( throw new DbException(e); } + /* If relation is broadcast, pick random worker to scan. */ + DistributeFunction df = getDatasetStatus(relationKey).getHowDistributed().getDf(); + if (df instanceof BroadcastDistributeFunction) { + scanWorkers = ImmutableSet.of(scanWorkers.iterator().next()); + } + /* Construct the operators that go elsewhere. */ DbQueryScan scan = new DbQueryScan(relationKey, schema); final ExchangePairID operatorId = ExchangePairID.newID(); From 366113a209ba0beabb3fdc3db18d2902bfafa799 Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Tue, 27 Dec 2016 22:14:27 -0800 Subject: [PATCH 04/15] omit workers equality check for broadcast relations --- .../myria/api/encoding/QueryConstruct.java | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index 97c09d123..fcc97f790 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -20,6 +20,7 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Sets; +import edu.washington.escience.myria.DbException; import edu.washington.escience.myria.MyriaConstants; import edu.washington.escience.myria.MyriaConstants.FTMode; import edu.washington.escience.myria.MyriaConstants.ProfilingMode; @@ -43,6 +44,8 @@ import edu.washington.escience.myria.operator.network.CollectProducer; import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.operator.network.EOSController; +import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; +import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.parallel.JsonSubQuery; import edu.washington.escience.myria.parallel.RelationWriteMetadata; @@ -66,7 +69,7 @@ public class QueryConstruct { */ public static Map instantiate( final List fragments, final ConstructArgs args) - throws CatalogException { + throws CatalogException, DbException { // Assign fragment index before everything else int idx = 0; @@ -146,6 +149,7 @@ public static void setQueryExecutionOptions( private static boolean setOrVerifyFragmentWorkers( @Nonnull final PlanFragmentEncoding fragment, @Nonnull final Collection workers, + final boolean isBroadcastScan, @Nonnull final String currentTask) { Preconditions.checkNotNull(fragment, "fragment"); Preconditions.checkNotNull(workers, "workers"); @@ -153,7 +157,7 @@ private static boolean setOrVerifyFragmentWorkers( if (fragment.workers == null) { fragment.workers = ImmutableList.copyOf(workers); return true; - } else { + } else if (!isBroadcastScan) { Preconditions.checkArgument( HashMultiset.create(fragment.workers).equals(HashMultiset.create(workers)), "During %s, cannot change workers for fragment %s from %s to %s", @@ -161,8 +165,8 @@ private static boolean setOrVerifyFragmentWorkers( fragment.fragmentIndex, fragment.workers, workers); - return false; } + return false; } /** @@ -177,7 +181,7 @@ private static boolean setOrVerifyFragmentWorkers( */ private static void setAndVerifyScans( final List fragments, final ConstructArgs args) - throws CatalogException { + throws CatalogException, DbException { Server server = args.getServer(); for (PlanFragmentEncoding fragment : fragments) { @@ -185,18 +189,18 @@ private static void setAndVerifyScans( Set scanWorkers; String scanRelation; + RelationKey relationKey = null; if (operator instanceof TableScanEncoding) { TableScanEncoding scan = ((TableScanEncoding) operator); - scanRelation = scan.relationKey.toString(); + relationKey = scan.relationKey; + scanRelation = relationKey.toString(); scanWorkers = server.getWorkersForRelation(scan.relationKey, scan.storedRelationId); } else if (operator instanceof TempTableScanEncoding) { TempTableScanEncoding scan = ((TempTableScanEncoding) operator); + relationKey = RelationKey.ofTemp(args.getQueryId(), scan.table); scanRelation = "temporary relation " + scan.table; scanWorkers = - server - .getQueryManager() - .getWorkersForTempRelation( - args.getQueryId(), RelationKey.ofTemp(args.getQueryId(), scan.table)); + server.getQueryManager().getWorkersForTempRelation(args.getQueryId(), relationKey); } else { continue; } @@ -206,7 +210,10 @@ private static void setAndVerifyScans( * Note: the current assumption is that all the partitions need to be scanned. This will not be true if we have * data replication, or allow to scan only a subset of the partitions. Revise if needed. */ - setOrVerifyFragmentWorkers(fragment, scanWorkers, "Setting workers for " + scanRelation); + DistributeFunction df = server.getDatasetStatus(relationKey).getHowDistributed().getDf(); + boolean isBroadcastScan = (df instanceof BroadcastDistributeFunction); + setOrVerifyFragmentWorkers( + fragment, scanWorkers, isBroadcastScan, "Setting workers for " + scanRelation); } } } @@ -340,7 +347,8 @@ private static void verifyAndPropagateLocalEdgeConstraints( // Verify that all fragments match the workers we found (and propagate if null) for (PlanFragmentEncoding frag : allFrags) { - anyUpdates |= setOrVerifyFragmentWorkers(frag, workers, "propagating edge constraints"); + anyUpdates |= + setOrVerifyFragmentWorkers(frag, workers, false, "propagating edge constraints"); } } } while (anyUpdates); @@ -466,7 +474,7 @@ private static void fillInRealOperatorAndWorkerIDs(final List fragments, final ConstructArgs args) - throws CatalogException { + throws CatalogException, DbException { /* 1. Honor user overrides. Note this is unchecked, but we may find constraint violations later. */ for (PlanFragmentEncoding fragment : fragments) { From 1e30def3728b66d3f41f015dbbd3bf8a0e0d0115 Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Fri, 30 Dec 2016 10:06:33 -0800 Subject: [PATCH 05/15] addressing comments --- .../encoding/BroadcastProducerEncoding.java | 5 +- .../myria/api/encoding/ConsumerEncoding.java | 6 +- .../api/encoding/EOSControllerEncoding.java | 7 +- .../GenericShuffleProducerEncoding.java | 5 +- .../HyperCubeShuffleProducerEncoding.java | 5 +- .../myria/api/encoding/OperatorEncoding.java | 14 +- .../myria/api/encoding/QueryConstruct.java | 44 +-- .../escience/myria/operator/DbInsert.java | 6 +- .../operator/network/CollectProducer.java | 16 +- .../network/LocalMultiwayProducer.java | 8 +- .../myria/operator/network/Producer.java | 2 +- .../operator/network/RecoverProducer.java | 22 +- .../distribute/DistributeFunction.java | 4 +- .../distribute/HashPartitionFunction.java | 11 +- .../network/distribute/HowDistributed.java | 16 +- .../HyperCubeDistributeFunction.java | 2 +- .../HyperCubePartitionFunction.java | 4 +- .../IdentityDistributeFunction.java | 3 +- .../distribute/IdentityPartitionFunction.java | 7 +- .../network/distribute/PartitionFunction.java | 2 +- .../RoundRobinPartitionFunction.java | 6 +- .../escience/myria/parallel/Server.java | 285 +++++++++++------- .../escience/myria/storage/TupleBatch.java | 85 ++++-- .../escience/myria/util/HashUtils.java | 2 +- .../escience/myria/util/MyriaArrayUtils.java | 3 +- .../escience/myria/util/MyriaUtils.java | 25 -- .../escience/myria/util/TestUtils.java | 34 ++- 27 files changed, 362 insertions(+), 267 deletions(-) diff --git a/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java index eb28946b0..2b19c614f 100644 --- a/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java @@ -1,10 +1,11 @@ package edu.washington.escience.myria.api.encoding; +import com.google.common.primitives.Ints; + import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.util.MyriaUtils; /** * JSON wrapper for BroadcastProducer @@ -16,7 +17,7 @@ public GenericShuffleProducer construct(ConstructArgs args) { return new GenericShuffleProducer( null, getRealOperatorIds().toArray(new ExchangePairID[] {}), - MyriaUtils.integerSetToIntArray(getRealWorkerIds()), + Ints.toArray(getRealWorkerIds()), new BroadcastDistributeFunction(getRealWorkerIds().size())); } } diff --git a/src/edu/washington/escience/myria/api/encoding/ConsumerEncoding.java b/src/edu/washington/escience/myria/api/encoding/ConsumerEncoding.java index 8ed2a78d1..19414aa1f 100644 --- a/src/edu/washington/escience/myria/api/encoding/ConsumerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/ConsumerEncoding.java @@ -1,5 +1,7 @@ package edu.washington.escience.myria.api.encoding; +import com.google.common.primitives.Ints; + import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.Consumer; import edu.washington.escience.myria.util.MyriaUtils; @@ -10,8 +12,6 @@ public class ConsumerEncoding extends AbstractConsumerEncoding { @Override public Consumer construct(ConstructArgs args) { return new Consumer( - null, - MyriaUtils.getSingleElement(getRealOperatorIds()), - MyriaUtils.integerSetToIntArray(getRealWorkerIds())); + null, MyriaUtils.getSingleElement(getRealOperatorIds()), Ints.toArray(getRealWorkerIds())); } } diff --git a/src/edu/washington/escience/myria/api/encoding/EOSControllerEncoding.java b/src/edu/washington/escience/myria/api/encoding/EOSControllerEncoding.java index aea013641..60ea8dbef 100644 --- a/src/edu/washington/escience/myria/api/encoding/EOSControllerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/EOSControllerEncoding.java @@ -2,10 +2,11 @@ import java.util.List; +import com.google.common.primitives.Ints; + import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.EOSController; import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.util.MyriaUtils; public class EOSControllerEncoding extends AbstractProducerEncoding { @@ -13,8 +14,6 @@ public class EOSControllerEncoding extends AbstractProducerEncoding ids = getRealOperatorIds(); return new EOSController( - null, - ids.toArray(new ExchangePairID[ids.size()]), - MyriaUtils.integerSetToIntArray(getRealWorkerIds())); + null, ids.toArray(new ExchangePairID[ids.size()]), Ints.toArray(getRealWorkerIds())); } } diff --git a/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java index cd23ca882..5d24236a7 100644 --- a/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java @@ -3,11 +3,12 @@ import java.util.List; import java.util.Set; +import com.google.common.primitives.Ints; + import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.util.MyriaUtils; /** JSON wrapper for GenericShuffleProducer encoding. */ public class GenericShuffleProducerEncoding @@ -28,7 +29,7 @@ public GenericShuffleProducer construct(final ConstructArgs args) { new GenericShuffleProducer( null, operatorIds.toArray(new ExchangePairID[] {}), - MyriaUtils.integerSetToIntArray(workerIds), + Ints.toArray(workerIds), distributeFunction); if (argBufferStateType != null) { producer.setBackupBuffer(argBufferStateType); diff --git a/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java index bc4457254..977c2e746 100644 --- a/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/HyperCubeShuffleProducerEncoding.java @@ -5,12 +5,13 @@ import javax.ws.rs.core.Response.Status; +import com.google.common.primitives.Ints; + import edu.washington.escience.myria.api.MyriaApiException; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.network.GenericShuffleProducer; import edu.washington.escience.myria.operator.network.distribute.HyperCubeDistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; -import edu.washington.escience.myria.util.MyriaUtils; /** * JSON Encoding for HyperCubeShuffle. @@ -26,7 +27,7 @@ public GenericShuffleProducer construct(ConstructArgs args) throws MyriaApiExcep return new GenericShuffleProducer( null, getRealOperatorIds().toArray(new ExchangePairID[] {}), - MyriaUtils.integerSetToIntArray( + Ints.toArray( args.getServer().getRandomWorkers(distributeFunction.getAllDestinations().size())), distributeFunction); } diff --git a/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java b/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java index 6ee54c962..b03a47bbd 100644 --- a/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/OperatorEncoding.java @@ -12,9 +12,11 @@ import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.Operator; -/** A JSON-able wrapper for the expected wire message for an operator. To add a new operator, two things need to be - * done. 1. Create an Encoding class that extends OperatorEncoding. 2. Add the operator to the list of (alphabetically - * sorted) JsonSubTypes below. */ +/** + * A JSON-able wrapper for the expected wire message for an operator. To add a new operator, two things need to be done. + * 1. Create an Encoding class that extends OperatorEncoding. + * 2. Add the operator to the list of (alphabetically sorted) JsonSubTypes below. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "opType") @JsonSubTypes({ @Type(name = "Aggregate", value = AggregateEncoding.class), @@ -81,7 +83,9 @@ public abstract class OperatorEncoding extends MyriaApiEncod /** Connect any operators to this one. */ public abstract void connect(Operator operator, Map operators); - /** @param args TODO - * @return an instantiated operator. */ + /** + * @param args construct args containing the server and query ID. + * @return an instantiated operator. + */ public abstract T construct(@Nonnull ConstructArgs args) throws MyriaApiException; } diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index 97c09d123..a76d950b9 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -61,6 +61,7 @@ public class QueryConstruct { * * @param fragments the JSON-encoded query fragments to be executed in parallel * @param server the server on which the query will be executed + * @param args construct args containing the server and query ID. * @return the physical plan * @throws CatalogException if there is an error instantiating the plan */ @@ -92,11 +93,9 @@ public static Map instantiate( } int loopCount = 0; while (setConsumerSchema(fragments, allOperators)) { - /* - * Do it iteratively until no new consumer has its schema to be set. Add a loop count to prevent us from having an + /* Do it iteratively until no new consumer has its schema to be set. Add a loop count to prevent us from having an * infinite loop (which should NOT happen). Since each iteration should set the schema of at least one consumer, - * setting the threshold to be the number of operators is enough. - */ + * setting the threshold to be the number of operators is enough. */ loopCount++; if (loopCount == allOperators.size()) { break; @@ -170,7 +169,6 @@ private static boolean setOrVerifyFragmentWorkers( * existing constraints. * * @see #assignWorkersToFragments(List, ConstructArgs) - * * @param fragments the fragments of the plan * @param args other arguments necessary for query construction * @throws CatalogException if there is an error getting information from the Catalog @@ -202,10 +200,8 @@ private static void setAndVerifyScans( } Preconditions.checkArgument( scanWorkers != null, "Unable to find workers that store %s", scanRelation); - /* - * Note: the current assumption is that all the partitions need to be scanned. This will not be true if we have - * data replication, or allow to scan only a subset of the partitions. Revise if needed. - */ + /* Note: the current assumption is that all the partitions need to be scanned. This will not be true if we have + * data replication, or allow to scan only a subset of the partitions. Revise if needed. */ setOrVerifyFragmentWorkers(fragment, scanWorkers, "Setting workers for " + scanRelation); } } @@ -216,7 +212,6 @@ private static void setAndVerifyScans( * support multiple consumers (LocalMultiwayProducer, EOSController) can have multiple consumers. * * @see #assignWorkersToFragments(List, ConstructArgs) - * * @param fragments the fragments of the plan */ public static void sanityCheckEdges(final List fragments) { @@ -281,7 +276,6 @@ public static void sanityCheckEdges(final List fragments) * LocalMultiwayProducers/Consumers with the same operator ID need to be assigned to the same set of workers. * * @see #assignWorkersToFragments(List, ConstructArgs) - * * @param fragments the fragments of the plan */ private static void verifyAndPropagateLocalEdgeConstraints( @@ -351,7 +345,6 @@ private static void verifyAndPropagateLocalEdgeConstraints( * existing constraints. * * @see #assignWorkersToFragments(List, ConstructArgs) - * * @param fragments the fragments of the plan * @param args other arguments necessary for query construction * @throws CatalogException if there is an error getting information from the Catalog @@ -397,7 +390,6 @@ private static void setAndVerifySingletonConstraints( * Actually allocate the real operator IDs and real worker IDs for the producers and consumers. * * @see #assignWorkersToFragments(List, ConstructArgs) - * * @param fragments the fragments of the plan */ private static void fillInRealOperatorAndWorkerIDs(final List fragments) { @@ -405,10 +397,8 @@ private static void fillInRealOperatorAndWorkerIDs(final List> producerWorkerMap = Maps.newHashMap(); Map> consumerWorkerMap = Maps.newHashMap(); - /* - * First pass: create a new ExchangePairID for each Consumer, and set it. Also track the workers for each producer - * and consumer. - */ + /* First pass: create a new ExchangePairID for each Consumer, and set it. Also track the workers for each producer + * and consumer. */ for (PlanFragmentEncoding fragment : fragments) { for (OperatorEncoding operator : fragment.operators) { if (operator instanceof AbstractConsumerEncoding) { @@ -447,9 +437,8 @@ private static void fillInRealOperatorAndWorkerIDs(final List *

  • Obey user-overrides of fragment workers.
  • *
  • Fragments that scan tables must use the workers that contain the data.
  • @@ -470,10 +459,8 @@ private static void assignWorkersToFragments( /* 1. Honor user overrides. Note this is unchecked, but we may find constraint violations later. */ for (PlanFragmentEncoding fragment : fragments) { - /* - * First, set it to be null since the fragment may have been instantiated in a previous iteration, but affected - * relations now may have different partitioning schemes than in the previous iteration. - */ + /* First, set it to be null since the fragment may have been instantiated in a previous iteration, but affected + * relations now may have different partitioning schemes than in the previous iteration. */ fragment.workers = null; if (fragment.overrideWorkers != null && fragment.overrideWorkers.size() > 0) { /* The workers are set in the plan. */ @@ -512,7 +499,7 @@ private static void assignWorkersToFragments( * Instantiate operators in the given fragment. * * @param planFragment the encoded plan fragment. - * @param args args + * @param args construct args containing the server and query ID. * @param allOperators a map to keep instantiated operators. */ private static void instantiateFragmentOperators( @@ -582,7 +569,6 @@ private static RootOperator instantiateFragment( } /** - * * @param fragments the JSON-encoded query fragments to be executed in parallel * @param allOperators a map to keep instantiated operators. * @return if any more consumer has its schema to be set. @@ -642,10 +628,8 @@ public static SubQuery getRelationTupleUpdateSubQuery( Preconditions.checkState( dbms != null, "Server must have a configured DBMS environment variable"); - /* - * Worker plans: for each relation, create a {@link DbQueryScan} to get the count, an {@link Apply} to add the - * {@link RelationKey}, then a {@link CollectProducer} to send the count to the master. - */ + /* Worker plans: for each relation, create a {@link DbQueryScan} to get the count, an {@link Apply} to add the + * {@link RelationKey}, then a {@link CollectProducer} to send the count to the master. */ Map workerPlans = Maps.newHashMap(); for (RelationWriteMetadata meta : relationsWritten.values()) { Set workers = meta.getWorkers(); diff --git a/src/edu/washington/escience/myria/operator/DbInsert.java b/src/edu/washington/escience/myria/operator/DbInsert.java index 033f738b2..182cf7800 100644 --- a/src/edu/washington/escience/myria/operator/DbInsert.java +++ b/src/edu/washington/escience/myria/operator/DbInsert.java @@ -147,8 +147,10 @@ public DbInsert( Preconditions.checkArgument( overwriteTable || indexes == null || indexes.size() == 0, "Cannot create indexes when appending to a relation."); - /* 1) construct immutable copies of the given indexes. 2) ensure that the index requests are valid: - lists of - * column references must be non-null. - column references are unique per index. */ + /* 1) construct immutable copies of the given indexes. + * 2) ensure that the index requests are valid: + * - lists of column references must be non-null. + * - column references are unique per index. */ if (indexes != null) { ImmutableList.Builder> index = ImmutableList.builder(); for (List i : indexes) { diff --git a/src/edu/washington/escience/myria/operator/network/CollectProducer.java b/src/edu/washington/escience/myria/operator/network/CollectProducer.java index 71835eee1..28e9d9d60 100644 --- a/src/edu/washington/escience/myria/operator/network/CollectProducer.java +++ b/src/edu/washington/escience/myria/operator/network/CollectProducer.java @@ -4,22 +4,26 @@ import edu.washington.escience.myria.operator.network.distribute.BroadcastDistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; -/** The producer part of the Collect Exchange operator. The producer actively pushes the tuples generated by the child - * operator to the paired CollectConsumer. */ +/** + * The producer part of the Collect Exchange operator. The producer actively pushes the tuples generated by the child + * operator to the paired CollectConsumer. + */ public class CollectProducer extends GenericShuffleProducer { /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** @param child the child who provides data for this producer to distribute. + /** + * @param child the child who provides data for this producer to distribute. * @param operatorID destination operator the data goes - * @param collectConsumerWorkerID destination worker the data goes. */ + * @param consumerWorkerID destination worker the data goes. + */ public CollectProducer( - final Operator child, final ExchangePairID operatorID, final int collectConsumerWorkerID) { + final Operator child, final ExchangePairID operatorID, final int consumerWorkerID) { super( child, new ExchangePairID[] {operatorID}, - new int[] {collectConsumerWorkerID}, + new int[] {consumerWorkerID}, new BroadcastDistributeFunction(1)); } } diff --git a/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java b/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java index 413b10beb..f9bea8b05 100644 --- a/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java +++ b/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java @@ -5,14 +5,16 @@ import edu.washington.escience.myria.parallel.ExchangePairID; import edu.washington.escience.myria.parallel.ipc.IPCConnectionPool; -/** A producer that duplicates tuple batches to corresponding LocalMultiwayConsumers. */ +/** A producer that duplicates tuple batches to corresponding consumers. */ public final class LocalMultiwayProducer extends GenericShuffleProducer { /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** @param child the child who provides data for this producer to distribute. - * @param operatorIDs destination operators the data goes */ + /** + * @param child the child who provides data for this producer to distribute. + * @param operatorIDs destination operators the data goes + */ public LocalMultiwayProducer(final Operator child, final ExchangePairID[] operatorIDs) { super( child, diff --git a/src/edu/washington/escience/myria/operator/network/Producer.java b/src/edu/washington/escience/myria/operator/network/Producer.java index f136bcfcb..bc5ff30ca 100644 --- a/src/edu/washington/escience/myria/operator/network/Producer.java +++ b/src/edu/washington/escience/myria/operator/network/Producer.java @@ -38,7 +38,7 @@ public abstract class Producer extends RootOperator implements StreamingStateful /** Required for Java serialization. */ private static final long serialVersionUID = 1L; - /** The worker this operator is located at. */ + /** The worker where this operator is located. */ private transient LocalFragmentResourceManager taskResourceManager; /** the netty channels doing the true IPC IO. */ diff --git a/src/edu/washington/escience/myria/operator/network/RecoverProducer.java b/src/edu/washington/escience/myria/operator/network/RecoverProducer.java index 5a72f6f6f..9625e76d5 100644 --- a/src/edu/washington/escience/myria/operator/network/RecoverProducer.java +++ b/src/edu/washington/escience/myria/operator/network/RecoverProducer.java @@ -9,8 +9,10 @@ import edu.washington.escience.myria.parallel.ipc.StreamOutputChannel; import edu.washington.escience.myria.storage.TupleBatch; -/** The producer part of the Collect Exchange operator. The producer actively pushes the tuples generated by the child - * operator to the paired CollectConsumer. */ +/** + * The producer part of the Collect Exchange operator. The producer actively pushes the tuples generated by the child + * operator to the paired consumer. + */ public final class RecoverProducer extends CollectProducer { /** Required for Java serialization. */ @@ -25,20 +27,22 @@ public final class RecoverProducer extends CollectProducer { /** the channel index that this operator is recovering for. */ private final int channelIndx; - /** @param child the child who provides data for this producer to distribute. + /** + * @param child the child who provides data for this producer to distribute. * @param operatorID destination operator the data goes - * @param collectConsumerWorkerID destination worker the data goes. + * @param consumerWorkerID destination worker the data goes. * @param oriProducer the original producer. - * @param channelIndx the channel index that this operator is recovering for. * */ + * @param channelInx the channel index that this operator is recovering for. * + */ public RecoverProducer( final Operator child, final ExchangePairID operatorID, - final int collectConsumerWorkerID, + final int consumerWorkerID, final Producer oriProducer, - final int channelIndx) { - super(child, operatorID, collectConsumerWorkerID); + final int channelInx) { + super(child, operatorID, consumerWorkerID); this.oriProducer = oriProducer; - this.channelIndx = channelIndx; + this.channelIndx = channelInx; } @Override diff --git a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java index c9dec02a0..9a4a58a24 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java @@ -15,8 +15,8 @@ /** * A dataset is distributed by two steps: First, using a partition function to generate a partition for each tuple; - * Second, mapping each partition to a set of destinations. A destination corresponds to an output channel, which is - * determined by a worker ID + an operator ID. + * Second, mapping each partition to a set of destinations. A destination corresponds to an output channel ID + * corresponding to a (worker ID, operator ID) pair. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonSubTypes({ diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java index db6ad2a66..22bdf7760 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java @@ -8,10 +8,10 @@ import edu.washington.escience.myria.storage.TupleBatch; import edu.washington.escience.myria.util.HashUtils; +import edu.washington.escience.myria.util.MyriaArrayUtils; /** - * Implementation that uses multiple fields as the key to hash The partition of a tuple is decided by the hash code of a - * group of fields of the tuple. + * The partition of a tuple is decided by the hash code of a group of fields of the tuple. */ public final class HashPartitionFunction extends PartitionFunction { @@ -49,6 +49,7 @@ public HashPartitionFunction(final int[] indexes, final int seedIndex) { i, indexes[i]); } + MyriaArrayUtils.checkSet(indexes); this.indexes = indexes; this.seedIndex = seedIndex % HashUtils.NUM_OF_HASHFUNCTIONS; } @@ -62,15 +63,15 @@ public int[] getIndexes() { @Override public TupleBatch[] partition(@Nonnull final TupleBatch tb) { - BitSet[] partitions = new BitSet[numPartition()]; + BitSet[] partitions = new BitSet[numPartitions()]; for (int i = 0; i < partitions.length; ++i) { partitions[i] = new BitSet(); } for (int i = 0; i < tb.numTuples(); i++) { - int p = Math.floorMod(HashUtils.hashSubRow(tb, indexes, i, seedIndex), numPartition()); + int p = Math.floorMod(HashUtils.hashSubRow(tb, indexes, i, seedIndex), numPartitions()); partitions[p].set(i); } - TupleBatch[] tbs = new TupleBatch[numPartition()]; + TupleBatch[] tbs = new TupleBatch[numPartitions()]; for (int i = 0; i < tbs.length; ++i) { tbs[i] = tb.filter(partitions[i]); } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java b/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java index 0d76dcb4c..f57c03a0c 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HowDistributed.java @@ -9,16 +9,18 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; -/** How is a dataset distributed. */ +/** How a dataset is distributed. */ public class HowDistributed { - /** The distribute function used to distirbute the dataset. Null means unknown. */ + /** The distribute function used to distribute the dataset. Null means unknown. */ @JsonProperty private DistributeFunction df = null; /** The sequence of workers that the dataset is partitioned on. Null means unknown. */ @JsonProperty private ImmutableList workers = null; - /** @param df the distribute function. - * @param workers the sequence of workers. */ + /** + * @param df the distribute function. + * @param workers the sequence of workers. + */ public HowDistributed(@Nullable final DistributeFunction df, @Nullable final int[] workers) { this.df = df; if (workers != null) { @@ -26,11 +28,13 @@ public HowDistributed(@Nullable final DistributeFunction df, @Nullable final int } } - /** Static function to create a HowPartitioned object. + /** + * Static function to create a HowPartitioned object. * * @param df the distribute function. * @param workers the sequence of workers. * - * @return a new HowPartitioned reference to the specified relation. */ + * @return a new HowPartitioned reference to the specified relation. + */ @JsonCreator public static HowDistributed of( @JsonProperty("df") final DistributeFunction df, diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java index b81ece120..ad3f11a1b 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java @@ -23,7 +23,7 @@ public HyperCubeDistributeFunction( @JsonProperty("hyperCubeDimensions") final int[] hyperCubeDimensions, @JsonProperty("hashedColumns") final int[] hashedColumns, @JsonProperty("mappedHCDimensions") final int[] mappedHCDimensions, - @JsonProperty("callPartition") final List> cellPartition) { + @JsonProperty("cellPartition") final List> cellPartition) { super( new HyperCubePartitionFunction(hyperCubeDimensions, hashedColumns, mappedHCDimensions), cellPartition); diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java index 4c14add19..e6bf73c1d 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubePartitionFunction.java @@ -34,7 +34,7 @@ public HyperCubePartitionFunction( @Override public TupleBatch[] partition(final TupleBatch tb) { - BitSet[] partitions = new BitSet[numPartition()]; + BitSet[] partitions = new BitSet[numPartitions()]; for (int i = 0; i < tb.numTuples(); i++) { int p = 0; for (int j = 0; j < hashedColumns.length; j++) { @@ -48,7 +48,7 @@ public TupleBatch[] partition(final TupleBatch tb) { } partitions[p].set(i); } - TupleBatch[] tbs = new TupleBatch[numPartition()]; + TupleBatch[] tbs = new TupleBatch[numPartitions()]; for (int i = 0; i < tbs.length; ++i) { tbs[i] = tb.filter(partitions[i]); } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java index c1e5cf3c0..930939d89 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java @@ -6,8 +6,7 @@ import edu.washington.escience.myria.util.MyriaArrayUtils; /** - * Implementation of a DistributeFunction that use the trivial identity hash. (i.e. a --> a) The attribute to hash on - * must be an INT column and should represent a workerID + * Implementation of a DistributeFunction that maps a tuple to a worker as specified in an INT column (i.e. a --> a). */ public final class IdentityDistributeFunction extends DistributeFunction { diff --git a/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java index 5df0141f6..ab1b64e03 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/IdentityPartitionFunction.java @@ -9,8 +9,7 @@ import edu.washington.escience.myria.storage.TupleBatch; /** - * Implementation that uses multiple fields as the key to hash The partition of - * a tuple is decided by the hash code of a group of fields of the tuple. + * Implementation of a PartitionFunction that maps a tuple to a partition as specified in an INT column (i.e. a --> a). */ public final class IdentityPartitionFunction extends PartitionFunction { @@ -29,11 +28,11 @@ public IdentityPartitionFunction(final int index) { @Override public TupleBatch[] partition(@Nonnull final TupleBatch tb) { - BitSet[] partitions = new BitSet[numPartition()]; + BitSet[] partitions = new BitSet[numPartitions()]; for (int i = 0; i < tb.numTuples(); i++) { partitions[tb.getInt(index, i) - 1].set(i); } - TupleBatch[] tbs = new TupleBatch[numPartition()]; + TupleBatch[] tbs = new TupleBatch[numPartitions()]; for (int i = 0; i < tbs.length; ++i) { tbs[i] = tb.filter(partitions[i]); } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java index c7fae59c1..e101c932a 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/PartitionFunction.java @@ -31,7 +31,7 @@ public abstract class PartitionFunction implements Serializable { private Integer numPartitions = null; /** @return the number of partitions. */ - public final int numPartition() { + public final int numPartitions() { Preconditions.checkState(numPartitions != null, "numPartitions has not been set"); return numPartitions; } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java index 5ca1ca115..1fd805f91 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java @@ -18,12 +18,12 @@ public final class RoundRobinPartitionFunction extends PartitionFunction { @Override public TupleBatch[] partition(@Nonnull final TupleBatch tb) { - BitSet[] partitions = new BitSet[numPartition()]; + BitSet[] partitions = new BitSet[numPartitions()]; for (int i = 0; i < tb.numTuples(); i++) { partitions[curPartition].set(i); - curPartition = (curPartition + 1) % numPartition(); + curPartition = (curPartition + 1) % numPartitions(); } - TupleBatch[] tbs = new TupleBatch[numPartition()]; + TupleBatch[] tbs = new TupleBatch[numPartitions()]; for (int i = 0; i < tbs.length; ++i) { tbs[i] = tb.filter(partitions[i]); } diff --git a/src/edu/washington/escience/myria/parallel/Server.java b/src/edu/washington/escience/myria/parallel/Server.java index 717e25b6f..a07952f6f 100644 --- a/src/edu/washington/escience/myria/parallel/Server.java +++ b/src/edu/washington/escience/myria/parallel/Server.java @@ -141,7 +141,6 @@ import edu.washington.escience.myria.tools.MyriaGlobalConfigurationModule.WorkerConf; import edu.washington.escience.myria.tools.MyriaWorkerConfigurationModule; import edu.washington.escience.myria.util.IPCUtils; -import edu.washington.escience.myria.util.MyriaUtils; import edu.washington.escience.myria.util.concurrent.ErrorLoggingTimerTask; import edu.washington.escience.myria.util.concurrent.RenamingThreadFactory; @@ -248,11 +247,9 @@ private void enqueueDriverMessage(@Nonnull final TransportMessage msg) { pendingDriverMessages.add(driverMsg); } - /* - * (non-Javadoc) + /* (non-Javadoc) * @see org.apache.reef.task.TaskMessageSource#getMessage() To be used to instruct the driver to launch or abort - * workers. - */ + * workers. */ @Override public Optional getMessage() { // TODO: determine which messages should be sent to the driver @@ -350,9 +347,11 @@ public QueryManager getQueryManager() { /** Execution environment variables for operators. */ private final ConcurrentHashMap execEnvVars; - /** All message queue. + /** + * All message queue. * - * @TODO remove this queue as in {@link Worker}s. */ + * @TODO remove this queue as in {@link Worker}s. + */ private final LinkedBlockingQueue> messageQueue; /** The IPC Connection Pool. */ @@ -364,8 +363,10 @@ public QueryManager getQueryManager() { /** The Catalog stores the metadata about the Myria instance. */ private MasterCatalog catalog; - /** The {@link OrderedMemoryAwareThreadPoolExecutor} who gets messages from {@link workerExecutor} and further process - * them using application specific message handlers, e.g. {@link MasterShortMessageProcessor}. */ + /** + * The {@link OrderedMemoryAwareThreadPoolExecutor} who gets messages from {@link workerExecutor} and further process + * them using application specific message handlers, e.g. {@link MasterShortMessageProcessor}. + */ private volatile OrderedMemoryAwareThreadPoolExecutor ipcPipelineExecutor; /** The {@link ExecutorService} who executes the master-side subqueries. */ @@ -417,7 +418,8 @@ QueryExecutionMode getExecutionMode() { private final int inputBufferRecoverTrigger; private final Injector injector; - /** Construct a server object, with configuration stored in the specified catalog file. + /** + * Construct a server object, with configuration stored in the specified catalog file. * * @param masterHost hostname of the master * @param masterPort RPC port of the master @@ -431,7 +433,8 @@ QueryExecutionMode getExecutionMode() { * @param inputBufferCapacity size of the input buffer in bytes * @param inputBufferRecoverTrigger number of bytes in the input buffer to trigger recovery after overflow * @param persistURI the storage endpoint URI for persisting partitioned relations - * @param injector a Tang injector for instantiating objects from configuration */ + * @param injector a Tang injector for instantiating objects from configuration + */ @Inject public Server( @Parameter(MasterHost.class) final String masterHost, @@ -528,10 +531,8 @@ private void cleanup() { scheduledTaskExecutor.shutdownNow(); } - /* - * Close the catalog before shutting down the IPC because there may be Catalog jobs pending that were triggered by - * IPC events. - */ + /* Close the catalog before shutting down the IPC because there may be Catalog jobs pending that were triggered by + * IPC events. */ catalog.close(); connectionPool.shutdown(); @@ -549,9 +550,11 @@ public void shutdown() { cleanup(); } - /** Start all the threads that do work for the server. + /** + * Start all the threads that do work for the server. * - * @throws Exception if any error occurs. */ + * @throws Exception if any error occurs. + */ public void start() throws Exception { LOGGER.info("Server starting on {}", masterSocketInfo); @@ -677,13 +680,15 @@ public void start() throws Exception { } } - /** Manually add a relation to the catalog. + /** + * Manually add a relation to the catalog. * * @param relationKey the relation to add * @param schema the schema of the relation to add * @param workers the workers that have the relation * @param force force add the relation; will replace an existing entry. - * @throws DbException if the catalog cannot be accessed */ + * @throws DbException if the catalog cannot be accessed + */ private void addRelationToCatalog( final RelationKey relationKey, final Schema schema, @@ -731,13 +736,15 @@ public String getDBMS() { return (String) execEnvVars.get(MyriaConstants.EXEC_ENV_VAR_DATABASE_SYSTEM); } - /** Can be only used in test. + /** + * Can be only used in test. * * @return true if the query plan is accepted and scheduled for execution. * @param masterRoot the root operator of the master plan * @param workerRoots the roots of the worker part of the plan, {workerID -> RootOperator[]} * @throws DbException if any error occurs. - * @throws CatalogException catalog errors. */ + * @throws CatalogException catalog errors. + */ public QueryFuture submitQueryPlan( final RootOperator masterRoot, final Map workerRoots) throws DbException, CatalogException { @@ -759,10 +766,12 @@ public Set getAliveWorkers() { return ImmutableSet.copyOf(aliveWorkers); } - /** Return a random subset of workers. + /** + * Return a random subset of workers. * * @param number the number of alive workers returned - * @return a subset of workers that are currently alive. */ + * @return a subset of workers that are currently alive. + */ public Set getRandomWorkers(final int number) { Preconditions.checkArgument( number <= getAliveWorkers().size(), @@ -780,7 +789,8 @@ public Map getWorkers() { return workers; } - /** Ingest the given dataset. + /** + * Ingest the given dataset. * * @param relationKey the name of the dataset. * @param workersToIngest restrict the workers to ingest data (null for all) @@ -789,7 +799,8 @@ public Map getWorkers() { * @param df the distribute function. * @return the status of the ingested dataset. * @throws InterruptedException interrupted - * @throws DbException if there is an error */ + * @throws DbException if there is an error + */ public DatasetStatus ingestDataset( final RelationKey relationKey, List workersToIngest, @@ -842,13 +853,15 @@ public DatasetStatus ingestDataset( return getDatasetStatus(relationKey); } - /** Parallel Ingest + /** + * Parallel Ingest * * @param relationKey the name of the dataset. * @param workersToIngest restrict the workers to ingest data (null for all) * @throws URIException * @throws DbException - * @throws InterruptedException */ + * @throws InterruptedException + */ public DatasetStatus parallelIngestDataset( final RelationKey relationKey, final Schema schema, @@ -869,7 +882,7 @@ public DatasetStatus parallelIngestDataset( int[] workersArray; if (workersToIngest == null) { - int[] allWorkers = MyriaUtils.integerSetToIntArray(getAliveWorkers()); + int[] allWorkers = Ints.toArray(getAliveWorkers()); int totalNumberOfWorkersToIngest = 0; for (int i = allWorkers.length; i >= 1; i--) { totalNumberOfWorkersToIngest = i; @@ -883,7 +896,7 @@ public DatasetStatus parallelIngestDataset( workersArray = Arrays.copyOfRange(allWorkers, 0, totalNumberOfWorkersToIngest); } else { Preconditions.checkArgument(actualWorkers.size() > 0, "Must use > 0 workers"); - workersArray = MyriaUtils.integerSetToIntArray(actualWorkers); + workersArray = Ints.toArray(actualWorkers); partitionSize = fileSize / workersArray.length; } Map workerPlans = new HashMap<>(); @@ -936,18 +949,18 @@ public DatasetStatus parallelIngestDataset( return getDatasetStatus(relationKey); } - /** @param relationKey the relationalKey of the dataset to import + /** + * @param relationKey the relationalKey of the dataset to import * @param schema the schema of the dataset to import * @param workersToImportFrom the set of workers * @throws DbException if there is an error - * @throws InterruptedException interrupted */ + * @throws InterruptedException interrupted + */ public void addDatasetToCatalog( final RelationKey relationKey, final Schema schema, final List workersToImportFrom) throws DbException, InterruptedException { - /* - * Figure out the workers we will use. If workersToImportFrom is null, use all active workers. - */ + /* Figure out the workers we will use. If workersToImportFrom is null, use all active workers. */ List actualWorkers = workersToImportFrom; if (workersToImportFrom == null) { actualWorkers = ImmutableList.copyOf(getWorkers().keySet()); @@ -977,10 +990,12 @@ public void addDatasetToCatalog( } } - /** @param relationKey the relationKey of the dataset to delete + /** + * @param relationKey the relationKey of the dataset to delete * @return the status * @throws DbException if there is an error - * @throws InterruptedException interrupted */ + * @throws InterruptedException interrupted + */ public void deleteDataset(final RelationKey relationKey) throws DbException, InterruptedException { @@ -1108,12 +1123,14 @@ public long createView( return queryID; } - /** Create a function and register it in the catalog + /** + * Create a function and register it in the catalog * * @param name the name of the function * @param definition the function definition (must be postgres specific) * @param outputSchema the output schema of the function - * @return the status of the function */ + * @return the status of the function + */ public String createFunction( final String name, final String definition, @@ -1173,10 +1190,12 @@ public String createFunction( return response; } - /** @param relationKey the relationKey of the dataset to persist + /** + * @param relationKey the relationKey of the dataset to persist * @return the queryID * @throws DbException if there is an error - * @throws InterruptedException interrupted */ + * @throws InterruptedException interrupted + */ public long persistDataset(final RelationKey relationKey) throws DbException, InterruptedException, URISyntaxException { long queryID; @@ -1227,16 +1246,20 @@ public long persistDataset(final RelationKey relationKey) return queryID; } - /** @param relationKey the key of the desired relation. + /** + * @param relationKey the key of the desired relation. * @return the schema of the specified relation, or null if not found. - * @throws CatalogException if there is an error getting the Schema out of the catalog. */ + * @throws CatalogException if there is an error getting the Schema out of the catalog. + */ public Schema getSchema(final RelationKey relationKey) throws CatalogException { return catalog.getSchema(relationKey); } - /** @param key the relation key. + /** + * @param key the relation key. * @param howPartitioned how the dataset was partitioned. - * @throws DbException if there is an catalog exception. */ + * @throws DbException if there is an catalog exception. + */ public void updateHowPartitioned(final RelationKey key, final HowDistributed howPartitioned) throws DbException { try { @@ -1246,18 +1269,22 @@ public void updateHowPartitioned(final RelationKey key, final HowDistributed how } } - /** @param relationKey the key of the desired relation. + /** + * @param relationKey the key of the desired relation. * @param storedRelationId indicates which copy of the desired relation we want to scan. * @return the list of workers that store the specified relation. - * @throws CatalogException if there is an error accessing the catalog. */ + * @throws CatalogException if there is an error accessing the catalog. + */ public Set getWorkersForRelation( final RelationKey relationKey, final Integer storedRelationId) throws CatalogException { return catalog.getWorkersForRelation(relationKey, storedRelationId); } - /** @param queryId the query that owns the desired temp relation. + /** + * @param queryId the query that owns the desired temp relation. * @param relationKey the key of the desired temp relation. - * @return the list of workers that store the specified relation. */ + * @return the list of workers that store the specified relation. + */ public Set getWorkersForTempRelation( @Nonnull final Long queryId, @Nonnull final RelationKey relationKey) { return queryManager.getQuery(queryId).getWorkersForTempRelation(relationKey); @@ -1268,8 +1295,10 @@ protected SocketInfo getSocketInfo() { return masterSocketInfo; } - /** @return A list of datasets in the system. - * @throws DbException if there is an error accessing the desired Schema. */ + /** + * @return A list of datasets in the system. + * @throws DbException if there is an error accessing the desired Schema. + */ public List getDatasets() throws DbException { try { return catalog.getDatasets(); @@ -1278,11 +1307,13 @@ public List getDatasets() throws DbException { } } - /** Get the metadata about a relation. + /** + * Get the metadata about a relation. * * @param relationKey specified which relation to get the metadata about. * @return the metadata of the specified relation. - * @throws DbException if there is an error getting the status. */ + * @throws DbException if there is an error getting the status. + */ public DatasetStatus getDatasetStatus(final RelationKey relationKey) throws DbException { try { return catalog.getDatasetStatus(relationKey); @@ -1291,9 +1322,11 @@ public DatasetStatus getDatasetStatus(final RelationKey relationKey) throws DbEx } } - /** @param searchTerm the search term + /** + * @param searchTerm the search term * @return the relations that match the search term - * @throws DbException if there is an error getting the relation keys. */ + * @throws DbException if there is an error getting the relation keys. + */ public List getMatchingRelationKeys(final String searchTerm) throws DbException { try { return catalog.getMatchingRelationKeys(searchTerm); @@ -1302,9 +1335,11 @@ public List getMatchingRelationKeys(final String searchTerm) throws } } - /** @param userName the user whose datasets we want to access. + /** + * @param userName the user whose datasets we want to access. * @return a list of datasets belonging to the specified user. - * @throws DbException if there is an error accessing the Catalog. */ + * @throws DbException if there is an error accessing the Catalog. + */ public List getDatasetsForUser(final String userName) throws DbException { try { return catalog.getDatasetsForUser(userName); @@ -1313,10 +1348,12 @@ public List getDatasetsForUser(final String userName) throws DbEx } } - /** @param userName the user whose datasets we want to access. + /** + * @param userName the user whose datasets we want to access. * @param programName the program by that user whose datasets we want to access. * @return a list of datasets belonging to the specified program. - * @throws DbException if there is an error accessing the Catalog. */ + * @throws DbException if there is an error accessing the Catalog. + */ public List getDatasetsForProgram(final String userName, final String programName) throws DbException { try { @@ -1326,9 +1363,11 @@ public List getDatasetsForProgram(final String userName, final St } } - /** @param queryId the id of the query. + /** + * @param queryId the id of the query. * @return a list of datasets belonging to the specified program. - * @throws DbException if there is an error accessing the Catalog. */ + * @throws DbException if there is an error accessing the Catalog. + */ public List getDatasetsForQuery(final int queryId) throws DbException { try { return catalog.getDatasetsForQuery(queryId); @@ -1337,27 +1376,33 @@ public List getDatasetsForQuery(final int queryId) throws DbExcep } } - /** @return the maximum query id that matches the search. + /** + * @return the maximum query id that matches the search. * @param searchTerm a token to match against the raw queries. If null, all queries match. - * @throws CatalogException if an error occurs */ + * @throws CatalogException if an error occurs + */ public long getMaxQuery(final String searchTerm) throws CatalogException { return catalog.getMaxQuery(searchTerm); } - /** @return the minimum query id that matches the search. + /** + * @return the minimum query id that matches the search. * @param searchTerm a token to match against the raw queries. If null, all queries match. - * @throws CatalogException if an error occurs */ + * @throws CatalogException if an error occurs + */ public long getMinQuery(final String searchTerm) throws CatalogException { return catalog.getMinQuery(searchTerm); } - /** Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. + /** + * Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. * * @param relationKey the relation to be downloaded. * @param writer the {@link TupleWriter} which will serialize the tuples. * @param dataSink the {@link DataSink} for the tuple destination * @return the query future from which the query status can be looked up. - * @throws DbException if there is an error in the system. */ + * @throws DbException if there is an error in the system. + */ public ListenableFuture startDataStream( final RelationKey relationKey, final TupleWriter writer, final DataSink dataSink) throws DbException { @@ -1403,13 +1448,15 @@ public ListenableFuture startDataStream( } } - /** Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. + /** + * Start a query that streams tuples from the specified relation to the specified {@link TupleWriter}. * * @param numTB the number of {@link TupleBatch}es to download from each worker. * @param writer the {@link TupleWriter} which will serialize the tuples. * @param dataSink the {@link DataSink} for the tuple destination * @return the query future from which the query status can be looked up. - * @throws DbException if there is an error in the system. */ + * @throws DbException if there is an error in the system. + */ public ListenableFuture startTestDataStream( final int numTB, final TupleWriter writer, final DataSink dataSink) throws DbException { @@ -1455,12 +1502,14 @@ public ListenableFuture startTestDataStream( } } - /** @param subqueryId the subquery id. + /** + * @param subqueryId the subquery id. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public ListenableFuture startSentLogDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -1549,17 +1598,17 @@ public ListenableFuture startSentLogDataStream( } } - /** Extracts the set of workers used in a saved, encoded physical plan. + /** + * Extracts the set of workers used in a saved, encoded physical plan. * * @param plan a {@link List}, cached during execution. - * @return the set of workers used during the execution of this subquery. */ + * @return the set of workers used during the execution of this subquery. + */ @Nonnull private Set getWorkersFromSubqueryPlan(final String plan) { - /* - * We need to accumulate the workers used in the plan. We could deserialize the plan as a + /* We need to accumulate the workers used in the plan. We could deserialize the plan as a * List... which it is, but for forwards and backwards compatiblity let's deserialize it as a - * List>... which it also is. - */ + * List>... which it also is. */ ObjectMapper mapper = MyriaJsonMapperProvider.getMapper(); List> fragments; Set actualWorkers = Sets.newHashSet(); @@ -1591,11 +1640,13 @@ private Set getWorkersFromSubqueryPlan(final String plan) { return actualWorkers; } - /** Returns the set of workers that executed a particular subquery. + /** + * Returns the set of workers that executed a particular subquery. * * @param subQueryId the subquery. * @return the set of workers that executed a particular subquery. - * @throws DbException if there is an error in the catalog. */ + * @throws DbException if there is an error in the catalog. + */ private Set getWorkersForSubQuery(final SubQueryId subQueryId) throws DbException { String serializedPlan; try { @@ -1608,11 +1659,13 @@ private Set getWorkersForSubQuery(final SubQueryId subQueryId) throws D return getWorkersFromSubqueryPlan(serializedPlan); } - /** @param subqueryId the subquery id. + /** + * @param subqueryId the subquery id. * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public ListenableFuture startAggregatedSentLogDataStream( final SubQueryId subqueryId, final TupleWriter writer, final DataSink dataSink) throws DbException { @@ -1690,7 +1743,8 @@ public ListenableFuture startAggregatedSentLogDataStream( } } - /** @param subqueryId the desired subquery. + /** + * @param subqueryId the desired subquery. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param start the earliest time where we need data * @param end the latest time @@ -1699,7 +1753,8 @@ public ListenableFuture startAggregatedSentLogDataStream( * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public QueryFuture startLogDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -1819,7 +1874,8 @@ public QueryFuture startLogDataStream( /** Upper bound on the number of bins a profiler can ask for. */ private static final long MAX_BINS = 10000; - /** @param subqueryId subquery id. + /** + * @param subqueryId subquery id. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param start start of the histogram * @param end the end of the histogram @@ -1828,7 +1884,8 @@ public QueryFuture startLogDataStream( * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public QueryFuture startHistogramDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -1939,12 +1996,14 @@ public QueryFuture startHistogramDataStream( } } - /** @param subqueryId the subquery id. + /** + * @param subqueryId the subquery id. * @param fragmentId the fragment id * @param writer writer to get data * @param dataSink the {@link DataSink} for the tuple destination * @return profiling logs for the query. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public QueryFuture startRangeDataStream( final SubQueryId subqueryId, final long fragmentId, @@ -2011,12 +2070,14 @@ public QueryFuture startRangeDataStream( } } - /** @param subqueryId subquery id. + /** + * @param subqueryId subquery id. * @param fragmentId the fragment id to return data for. All fragments, if < 0. * @param writer writer to get data. * @param dataSink the {@link DataSink} for the tuple destination * @return contributions for operator. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public QueryFuture startContributionsStream( final SubQueryId subqueryId, final long fragmentId, @@ -2092,11 +2153,13 @@ public QueryFuture startContributionsStream( } } - /** Update the {@link MasterCatalog} so that the specified relation has the specified tuple count. + /** + * Update the {@link MasterCatalog} so that the specified relation has the specified tuple count. * * @param relation the relation to update * @param count the number of tuples in that relation - * @throws DbException if there is an error in the catalog */ + * @throws DbException if there is an error in the catalog + */ public void updateRelationTupleCount(final RelationKey relation, final long count) throws DbException { try { @@ -2106,11 +2169,13 @@ public void updateRelationTupleCount(final RelationKey relation, final long coun } } - /** Set the global variable owned by the specified query and named by the specified key to the specified value. + /** + * Set the global variable owned by the specified query and named by the specified key to the specified value. * * @param queryId the query to whom the variable belongs. * @param key the name of the variable - * @param value the new value for the variable */ + * @param value the new value for the variable + */ public void setQueryGlobal( final long queryId, @Nonnull final String key, @Nonnull final Object value) { Preconditions.checkNotNull(key, "key"); @@ -2118,29 +2183,35 @@ public void setQueryGlobal( queryManager.getQuery(queryId).setGlobal(key, value); } - /** Get the value of global variable owned by the specified query and named by the specified key. + /** + * Get the value of global variable owned by the specified query and named by the specified key. * * @param queryId the query to whom the variable belongs. * @param key the name of the variable - * @return the value of the variable */ + * @return the value of the variable + */ @Nullable public Object getQueryGlobal(final long queryId, @Nonnull final String key) { Preconditions.checkNotNull(key, "key"); return queryManager.getQuery(queryId).getGlobal(key); } - /** Return the schema of the specified temp relation in the specified query. + /** + * Return the schema of the specified temp relation in the specified query. * * @param queryId the query that owns the temp relation * @param name the name of the temporary relation - * @return the schema of the specified temp relation in the specified query */ + * @return the schema of the specified temp relation in the specified query + */ public Schema getTempSchema(@Nonnull final Long queryId, @Nonnull final String name) { return queryManager.getQuery(queryId).getTempSchema(RelationKey.ofTemp(queryId, name)); } - /** @param queryId the query id to fetch + /** + * @param queryId the query id to fetch * @param writerOutput the output stream to write results to. - * @throws DbException if there is an error in the database. */ + * @throws DbException if there is an error in the database. + */ public void getResourceUsage(final long queryId, final DataSink dataSink) throws DbException { Schema schema = Schema.appendColumn(MyriaConstants.RESOURCE_PROFILING_SCHEMA, Type.INT_TYPE, "workerId"); @@ -2160,10 +2231,12 @@ public void getResourceUsage(final long queryId, final DataSink dataSink) throws } } - /** @param queryId query id. + /** + * @param queryId query id. * @param writer writer to get data. * @return resource logs for the query. - * @throws DbException if there is an error when accessing profiling logs. */ + * @throws DbException if there is an error when accessing profiling logs. + */ public ListenableFuture getResourceLog( final long queryId, final TupleWriter writer, final DataSink dataSink) throws DbException { SubQueryId sqId = new SubQueryId(queryId, 0); @@ -2218,11 +2291,13 @@ public ListenableFuture getResourceLog( } } - /** Record the fact that this subquery executed this in the catalog. + /** + * Record the fact that this subquery executed this in the catalog. * * @param subQueryId the id of the subquery. * @param encodedPlan the plan. - * @throws DbException if there is an error in the catalog. */ + * @throws DbException if there is an error in the catalog. + */ public void setQueryPlan(final SubQueryId subQueryId, @Nonnull final String encodedPlan) throws DbException { try { @@ -2232,9 +2307,11 @@ public void setQueryPlan(final SubQueryId subQueryId, @Nonnull final String enco } } - /** @param subQueryId the query whose plan to look up. + /** + * @param subQueryId the query whose plan to look up. * @return the execution plan for this query. - * @throws DbException if there is an error getting the query status. */ + * @throws DbException if there is an error getting the query status. + */ @Nullable public String getQueryPlan(@Nonnull final SubQueryId subQueryId) throws DbException { try { diff --git a/src/edu/washington/escience/myria/storage/TupleBatch.java b/src/edu/washington/escience/myria/storage/TupleBatch.java index 12a225c41..70c6d8463 100644 --- a/src/edu/washington/escience/myria/storage/TupleBatch.java +++ b/src/edu/washington/escience/myria/storage/TupleBatch.java @@ -20,7 +20,10 @@ import edu.washington.escience.myria.util.IPCUtils; import net.jcip.annotations.ThreadSafe; -/** Container class for a batch of tuples. The goal is to amortize memory management overhead. */ +/** + * Container class for a batch of tuples. The goal is to amortize memory management overhead and processing overhead + * from code/data locality. + */ @ThreadSafe public class TupleBatch implements ReadableTable, Serializable { /** Required for Java serialization. */ @@ -36,10 +39,12 @@ public class TupleBatch implements ReadableTable, Serializable { /** Whether this TB is an EOI TB. */ private final boolean isEOI; - /** EOI TB constructor. + /** + * EOI TB constructor. * * @param schema schema of the tuples in this batch. - * @param isEoi whether this TupleBatch is an EOI TupleBatch. */ + * @param isEoi whether this TupleBatch is an EOI TupleBatch. + */ private TupleBatch(final Schema schema, final boolean isEoi) { this.schema = schema; numTuples = 0; @@ -51,38 +56,46 @@ private TupleBatch(final Schema schema, final boolean isEoi) { isEOI = isEoi; } - /** @param columnNames the new column names. - * @return a shallow copy of the specified TupleBatch with the new column names. */ + /** + * @param columnNames the new column names. + * @return a shallow copy of the specified TupleBatch with the new column names. + */ public TupleBatch rename(final List columnNames) { Schema newSchema = Schema.of(schema.getColumnTypes(), Objects.requireNonNull(columnNames, "columnNames")); return new TupleBatch(newSchema, columns, numTuples, isEOI); } - /** Standard immutable TupleBatch constructor. All fields must be populated before creation and cannot be changed. + /** + * Standard immutable TupleBatch constructor. All fields must be populated before creation and cannot be changed. * * @param schema schema of the tuples in this batch. Must match columns. * @param columns contains the column-stored data. Must match schema. - * @param numTuples the number of tuples in this TupleBatch. */ + * @param numTuples the number of tuples in this TupleBatch. + */ public TupleBatch( final Schema schema, final List> columns, final int numTuples) { this(schema, columns, numTuples, false); } - /** Constructor that gets the number of tuples from the columns. + /** + * Constructor that gets the number of tuples from the columns. * * @param schema schema of the tuples in this batch. Must match columns. - * @param columns contains the column-stored data. Must match schema. */ + * @param columns contains the column-stored data. Must match schema. + */ public TupleBatch(final Schema schema, final List> columns) { this(schema, columns, columns.get(0).size()); } - /** Construct a TupleBatch from the specified components. + /** + * Construct a TupleBatch from the specified components. * * @param schema schema of the tuples in this batch. Must match columns. * @param columns schema of the tuples in this batch. Must match columns. * @param numTuples the number of tuples in this batch. Must match columns. - * @param isEOI whether this is an EOI TupleBatch. */ + * @param isEOI whether this is an EOI TupleBatch. + */ public TupleBatch( final Schema schema, final List> columns, @@ -106,9 +119,11 @@ public TupleBatch( this.isEOI = isEOI; } - /** put the tuple batch into TBB by smashing it into cells and putting them one by one. + /** + * put the tuple batch into TBB by smashing it into cells and putting them one by one. * - * @param tbb the TBB buffer. */ + * @param tbb the TBB buffer. + */ public final void compactInto(final TupleBatchBuffer tbb) { if (isEOI()) { /* an EOI TB has no data */ @@ -120,12 +135,14 @@ public final void compactInto(final TupleBatchBuffer tbb) { } } - /** Return a new TupleBatch that contains only the filtered rows of the current dataset. Note that if some of the + /** + * Return a new TupleBatch that contains only the filtered rows of the current dataset. Note that if some of the * tuples in this batch are invalid, we will have to map the indices in the specified filter to the "real" indices in * the tuple. * * @param filter the rows to be retained. - * @return a TupleBatch that contains only the filtered rows of the current dataset. */ + * @return a TupleBatch that contains only the filtered rows of the current dataset. + */ public final TupleBatch filter(final BitSet filter) { Preconditions.checkArgument( filter.length() <= numTuples(), @@ -146,10 +163,12 @@ public final TupleBatch filter(final BitSet filter) { return new TupleBatch(schema, newColumns.build(), newNumTuples, isEOI); } - /** Return a new TupleBatch that contains only first prefix rows of this batch. + /** + * Return a new TupleBatch that contains only first prefix rows of this batch. * * @param prefix the number of rows in the prefix to be retained. - * @return a TupleBatch that contains only the filtered rows of the current dataset. */ + * @return a TupleBatch that contains only the filtered rows of the current dataset. + */ @SuppressWarnings({"rawtypes", "unchecked"}) public final TupleBatch prefix(final int prefix) { Preconditions.checkArgument( @@ -220,25 +239,29 @@ public final int numTuples() { return numTuples; } - /** Partition this TB using the partition function. The method is implemented by shallow copy of TupleBatches. + /** + * Partition this TB using the partition function. The method is implemented by shallow copy of TupleBatches. * * @return an array of TBs. The length of the array is the same as the number of partitions. If no tuple presents in a * partition, say the i'th partition, the i'th element in the result array is null. - * @param pf the partition function. */ + * @param pf the partition function. + */ public final TupleBatch[] partition(final PartitionFunction pf) { if (isEOI) { - TupleBatch[] result = new TupleBatch[pf.numPartition()]; + TupleBatch[] result = new TupleBatch[pf.numPartitions()]; Arrays.fill(result, this); return result; } return pf.partition(this); } - /** Creates a new TupleBatch with only the indicated columns. Internal implementation of a (non-duplicate-eliminating) + /** + * Creates a new TupleBatch with only the indicated columns. Internal implementation of a (non-duplicate-eliminating) * PROJECT statement. * * @param remainingColumns zero-indexed array of columns to retain. - * @return a projected TupleBatch. */ + * @return a projected TupleBatch. + */ public final TupleBatch selectColumns(final int[] remainingColumns) { Objects.requireNonNull(remainingColumns); final ImmutableList.Builder> newColumns = new ImmutableList.Builder>(); @@ -249,8 +272,10 @@ public final TupleBatch selectColumns(final int[] remainingColumns) { getSchema().getSubSchema(remainingColumns), newColumns.build(), numTuples, isEOI); } - /** @param rows a BitSet flagging the rows to be removed. - * @return a new TB with the specified rows removed. */ + /** + * @param rows a BitSet flagging the rows to be removed. + * @return a new TB with the specified rows removed. + */ public final TupleBatch filterOut(final BitSet rows) { BitSet inverted = (BitSet) rows.clone(); inverted.flip(0, numTuples); @@ -285,10 +310,12 @@ public final TransportMessage toTransportMessage() { return IPCUtils.normalDataMessage(columns, numTuples); } - /** Create an EOI TupleBatch. + /** + * Create an EOI TupleBatch. * * @param schema schema. - * @return EOI TB for the schema. */ + * @return EOI TB for the schema. + */ public static final TupleBatch eoiTupleBatch(final Schema schema) { return new TupleBatch(schema, true); } @@ -298,12 +325,14 @@ public final boolean isEOI() { return isEOI; } - /** Construct a new TupleBatch that equals the current batch with the specified column appended. The number of valid + /** + * Construct a new TupleBatch that equals the current batch with the specified column appended. The number of valid * tuples in this batch must be the same as the size of the other batch. If this batch is not dense, then * * @param columnName the name of the column to be added. * @param column the column to be added. - * @return a new TupleBatch containing the tuples of this column plus the tuples of the other. */ + * @return a new TupleBatch containing the tuples of this column plus the tuples of the other. + */ public TupleBatch appendColumn(final String columnName, final Column column) { Preconditions.checkArgument( numTuples() == column.size(), diff --git a/src/edu/washington/escience/myria/util/HashUtils.java b/src/edu/washington/escience/myria/util/HashUtils.java index 8d4e87b67..615930755 100644 --- a/src/edu/washington/escience/myria/util/HashUtils.java +++ b/src/edu/washington/escience/myria/util/HashUtils.java @@ -74,7 +74,7 @@ public static int hashValue(final ReadableTable table, final int column, final i * @param table the table containing the values to be hashed * @param column the column containing the value to be hashed * @param row the row containing the value to be hashed - * @param seedIndex the index of the chosen hashcode + * @param seedIndex the index of the chosen hash function * @return hash code of the specified seed */ public static int hashValue( diff --git a/src/edu/washington/escience/myria/util/MyriaArrayUtils.java b/src/edu/washington/escience/myria/util/MyriaArrayUtils.java index c315ad856..2f50b7f0d 100644 --- a/src/edu/washington/escience/myria/util/MyriaArrayUtils.java +++ b/src/edu/washington/escience/myria/util/MyriaArrayUtils.java @@ -9,11 +9,10 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; -import jersey.repackaged.com.google.common.collect.Lists; - /** * Array related utility functions. */ diff --git a/src/edu/washington/escience/myria/util/MyriaUtils.java b/src/edu/washington/escience/myria/util/MyriaUtils.java index 59a9d977a..2d6724c4f 100644 --- a/src/edu/washington/escience/myria/util/MyriaUtils.java +++ b/src/edu/washington/escience/myria/util/MyriaUtils.java @@ -4,8 +4,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; import org.joda.time.DateTime; @@ -13,7 +11,6 @@ /** * Generic utilities for Myria. - * */ public final class MyriaUtils { /** @@ -51,28 +48,6 @@ public static T getSingleElement(final Set input) { return null; } - /** - * Convert a collection of integers to a sorted int[]. - * - * @param input the collection of integers. - * @return an int[] containing the given integers. - */ - public static int[] integerSetToIntArray(final Set input) { - SortedSet set; - if (input instanceof SortedSet) { - set = (SortedSet) input; - } else { - set = new TreeSet<>(input); - } - int[] output = new int[input.size()]; - int i = 0; - for (int value : set) { - output[i] = value; - ++i; - } - return output; - } - /** * Helper function that generates an array of the numbers 0..max-1. * diff --git a/test/edu/washington/escience/myria/util/TestUtils.java b/test/edu/washington/escience/myria/util/TestUtils.java index bb72caed3..b1799fa86 100644 --- a/test/edu/washington/escience/myria/util/TestUtils.java +++ b/test/edu/washington/escience/myria/util/TestUtils.java @@ -58,9 +58,11 @@ public int compare(final Entry o1, final Entry o2) { private static Random random = null; - /** See http://docs.travis-ci.com/user/ci-environment/#Environment-variables + /** + * See http://docs.travis-ci.com/user/ci-environment/#Environment-variables * - * @return true if the system is currently in a Travis CI build. */ + * @return true if the system is currently in a Travis CI build. + */ public static boolean inTravis() { String travis = System.getenv("TRAVIS"); return (travis != null) && travis.equals("true"); @@ -414,10 +416,12 @@ public static HashMap tupleBatchToTupleBag(final TupleBatchBuffe return result; } - /** @param numTuples how many tuples in output + /** + * @param numTuples how many tuples in output * @param sampleSize how many different values should be created at random (around numTuples/sampleSize duplicates) * @param sorted Generate sorted tuples, sorted by id - * @return */ + * @return + */ public static TupleBatchBuffer generateRandomTuples( final int numTuples, final int sampleSize, final boolean sorted) { final ArrayList> entries = new ArrayList>(); @@ -447,15 +451,17 @@ public static TupleBatchBuffer generateRandomTuples( return tbb; } - /** Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the + /** + * Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the * specified relation key and partition function. * * @param masterSource the source of tuples, from the master. * @param dest the name of the relation into which tuples will be inserted (using overwrite!). - * @param df + * @param df how tuples will be distributed on the cluster. * @param workers the set of workers on which the data will be stored. * @return a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the - * specified relation key and partition function. */ + * specified relation key and partition function. + */ public static final SubQuery insertRelation( @Nonnull final Operator masterSource, @Nonnull final RelationKey dest, @@ -468,15 +474,17 @@ public static final SubQuery insertRelation( ArrayUtils.toPrimitive(workers.toArray(new Integer[workers.size()]))); } - /** Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the + /** + * Construct a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the * specified relation key and partition function. * * @param masterSource the source of tuples, from the master. * @param dest the name of the relation into which tuples will be inserted (using overwrite!). - * @param df + * @param df how tuples will be distributed on the cluster. * @param workers the set of workers on which the data will be stored. * @return a SubQuery that will insert the given tuples (starting on the master) on the specified workers using the - * specified relation key and partition function. */ + * specified relation key and partition function. + */ public static final SubQuery insertRelation( @Nonnull final Operator masterSource, @Nonnull final RelationKey dest, @@ -527,11 +535,13 @@ public static final SubQuery failOnFirstWorkerInit(@Nonnull final int[] workers) return new SubQuery(masterPlan, workerPlans); } - /** Returns a {@link TupleBatchBuffer} containing the values 0 to {@code n-1}. The column is of type + /** + * Returns a {@link TupleBatchBuffer} containing the values 0 to {@code n-1}. The column is of type * {@Link Type#INT_TYPE} and the column name is {@code "val"}. * * @param n the number of values in the buffer. - * @return a {@link TupleBatchBuffer} containing the values 0 to {@code n-1} */ + * @return a {@link TupleBatchBuffer} containing the values 0 to {@code n-1} + */ public static TupleBatchBuffer range(final int n) { TupleBatchBuffer sourceBuffer = new TupleBatchBuffer(Schema.ofFields(Type.INT_TYPE, "val")); for (int i = 0; i < n; ++i) { From 960132e74ae15cd8652d497bd0c38247d9675ad9 Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Fri, 30 Dec 2016 10:17:37 -0800 Subject: [PATCH 06/15] fix formatting and javadoc --- .../myria/api/encoding/QueryConstruct.java | 213 +++-- .../escience/myria/parallel/Server.java | 862 +++++++++++++----- .../escience/myria/util/HashUtils.java | 2 +- 3 files changed, 760 insertions(+), 317 deletions(-) diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index 86e5f3a09..8e7ba795d 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -55,7 +55,8 @@ public class QueryConstruct { - private static final org.slf4j.Logger LOGGER = org.slf4j.LoggerFactory.getLogger(QueryConstruct.class); + private static final org.slf4j.Logger LOGGER = + org.slf4j.LoggerFactory.getLogger(QueryConstruct.class); /** * Instantiate the server's desired physical plan from a list of JSON encodings of fragments. This list must contain a @@ -67,8 +68,9 @@ public class QueryConstruct { * @return the physical plan * @throws CatalogException if there is an error instantiating the plan */ - public static Map instantiate(final List fragments, - final ConstructArgs args) throws CatalogException, DbException { + public static Map instantiate( + final List fragments, final ConstructArgs args) + throws CatalogException, DbException { // Assign fragment index before everything else int idx = 0; @@ -124,7 +126,9 @@ public static Map instantiate(final List plans, final FTMode ftMode, + public static void setQueryExecutionOptions( + final Map plans, + final FTMode ftMode, @Nonnull final Set profilingMode) { for (SubQueryPlan plan : plans.values()) { plan.setFTMode(ftMode); @@ -141,8 +145,11 @@ public static void setQueryExecutionOptions(final Map pla * @return true if the workers were newly assigned * @throws IllegalArgumentException if the fragment already has workers, and the new set does not match */ - private static boolean setOrVerifyFragmentWorkers(@Nonnull final PlanFragmentEncoding fragment, - @Nonnull final Collection workers, final boolean isBroadcastScan, @Nonnull final String currentTask) { + private static boolean setOrVerifyFragmentWorkers( + @Nonnull final PlanFragmentEncoding fragment, + @Nonnull final Collection workers, + final boolean isBroadcastScan, + @Nonnull final String currentTask) { Preconditions.checkNotNull(fragment, "fragment"); Preconditions.checkNotNull(workers, "workers"); Preconditions.checkNotNull(currentTask, "currentTask"); @@ -150,9 +157,13 @@ private static boolean setOrVerifyFragmentWorkers(@Nonnull final PlanFragmentEnc fragment.workers = ImmutableList.copyOf(workers); return true; } else if (!isBroadcastScan) { - Preconditions.checkArgument(HashMultiset.create(fragment.workers).equals(HashMultiset.create(workers)), - "During %s, cannot change workers for fragment %s from %s to %s", currentTask, fragment.fragmentIndex, - fragment.workers, workers); + Preconditions.checkArgument( + HashMultiset.create(fragment.workers).equals(HashMultiset.create(workers)), + "During %s, cannot change workers for fragment %s from %s to %s", + currentTask, + fragment.fragmentIndex, + fragment.workers, + workers); } return false; } @@ -166,7 +177,8 @@ private static boolean setOrVerifyFragmentWorkers(@Nonnull final PlanFragmentEnc * @param args other arguments necessary for query construction * @throws CatalogException if there is an error getting information from the Catalog */ - private static void setAndVerifyScans(final List fragments, final ConstructArgs args) + private static void setAndVerifyScans( + final List fragments, final ConstructArgs args) throws CatalogException, DbException { Server server = args.getServer(); @@ -185,14 +197,17 @@ private static void setAndVerifyScans(final List fragments TempTableScanEncoding scan = ((TempTableScanEncoding) operator); relationKey = RelationKey.ofTemp(args.getQueryId(), scan.table); scanRelation = "temporary relation " + scan.table; - scanWorkers = server.getQueryManager().getWorkersForTempRelation(args.getQueryId(), relationKey); + scanWorkers = + server.getQueryManager().getWorkersForTempRelation(args.getQueryId(), relationKey); } else { continue; } - Preconditions.checkArgument(scanWorkers != null, "Unable to find workers that store %s", scanRelation); + Preconditions.checkArgument( + scanWorkers != null, "Unable to find workers that store %s", scanRelation); DistributeFunction df = server.getDatasetStatus(relationKey).getHowDistributed().getDf(); boolean isBroadcastScan = (df instanceof BroadcastDistributeFunction); - setOrVerifyFragmentWorkers(fragment, scanWorkers, isBroadcastScan, "Setting workers for " + scanRelation); + setOrVerifyFragmentWorkers( + fragment, scanWorkers, isBroadcastScan, "Setting workers for " + scanRelation); } } } @@ -218,15 +233,20 @@ public static void sanityCheckEdges(final List fragments) if (operator instanceof AbstractConsumerEncoding) { AbstractConsumerEncoding consumer = (AbstractConsumerEncoding) operator; consumerMap.put(consumer.argOperatorId, fragment); - } else if (operator instanceof AbstractProducerEncoding || operator instanceof IDBControllerEncoding) { + } else if (operator instanceof AbstractProducerEncoding + || operator instanceof IDBControllerEncoding) { Integer opId = operator.opId; PlanFragmentEncoding oldFragment = producerMap.put(opId, fragment); if (oldFragment != null) { - Preconditions.checkArgument(false, - "Two different operators cannot produce the same opId %s. Fragments: %s %s", opId, - fragment.fragmentIndex, oldFragment.fragmentIndex); + Preconditions.checkArgument( + false, + "Two different operators cannot produce the same opId %s. Fragments: %s %s", + opId, + fragment.fragmentIndex, + oldFragment.fragmentIndex); } - if (!(operator instanceof LocalMultiwayProducerEncoding || operator instanceof EOSControllerEncoding)) { + if (!(operator instanceof LocalMultiwayProducerEncoding + || operator instanceof EOSControllerEncoding)) { soleConsumer.add(opId); } } @@ -235,16 +255,23 @@ public static void sanityCheckEdges(final List fragments) /* Sanity check 1: Producer must have corresponding consumers, and vice versa. */ Set consumedNotProduced = Sets.difference(consumerMap.keySet(), producerMap.keySet()); - Preconditions.checkArgument(consumedNotProduced.isEmpty(), "Missing producer(s) for consumer(s): %s", + Preconditions.checkArgument( + consumedNotProduced.isEmpty(), + "Missing producer(s) for consumer(s): %s", consumedNotProduced); Set producedNotConsumed = Sets.difference(producerMap.keySet(), consumerMap.keySet()); - Preconditions.checkArgument(producedNotConsumed.isEmpty(), "Missing consumer(s) for producer(s): %s", + Preconditions.checkArgument( + producedNotConsumed.isEmpty(), + "Missing consumer(s) for producer(s): %s", producedNotConsumed); /* Sanity check 2: Operators that only admit a single consumer should have exactly one consumer. */ for (Integer opId : soleConsumer) { Collection consumers = consumerMap.get(opId); - Preconditions.checkArgument(consumers.size() == 1, "Producer %s only supports a single consumer, not %s", opId, + Preconditions.checkArgument( + consumers.size() == 1, + "Producer %s only supports a single consumer, not %s", + opId, consumers.size()); } } @@ -256,7 +283,8 @@ public static void sanityCheckEdges(final List fragments) * @see #assignWorkersToFragments(List, ConstructArgs) * @param fragments the fragments of the plan */ - private static void verifyAndPropagateLocalEdgeConstraints(final List fragments) { + private static void verifyAndPropagateLocalEdgeConstraints( + final List fragments) { // producers must be unique Map producerMap = Maps.newHashMap(); // consumers can be repeated, as long as the producer is a LocalMultiwayProducer @@ -277,10 +305,14 @@ private static void verifyAndPropagateLocalEdgeConstraints(final List consumedNotProduced = Sets.difference(consumerMap.keySet(), producerMap.keySet()); - Preconditions.checkArgument(consumedNotProduced.isEmpty(), "Missing LocalMultiwayProducer(s) for consumer(s): %s", + Preconditions.checkArgument( + consumedNotProduced.isEmpty(), + "Missing LocalMultiwayProducer(s) for consumer(s): %s", consumedNotProduced); Set producedNotConsumed = Sets.difference(producerMap.keySet(), consumerMap.keySet()); - Preconditions.checkArgument(producedNotConsumed.isEmpty(), "Missing LocalMultiwayConsumer(s) for producer(s): %s", + Preconditions.checkArgument( + producedNotConsumed.isEmpty(), + "Missing LocalMultiwayConsumer(s) for producer(s): %s", producedNotConsumed); boolean anyUpdates; @@ -307,7 +339,8 @@ private static void verifyAndPropagateLocalEdgeConstraints(final List fragments, - final ConstructArgs args) { - List singletonWorkers = ImmutableList.of(args.getServer().getAliveWorkers().iterator().next()); + private static void setAndVerifySingletonConstraints( + final List fragments, final ConstructArgs args) { + List singletonWorkers = + ImmutableList.of(args.getServer().getAliveWorkers().iterator().next()); for (PlanFragmentEncoding fragment : fragments) { for (OperatorEncoding operator : fragment.operators) { - if (operator instanceof CollectConsumerEncoding || operator instanceof SingletonEncoding - || operator instanceof EOSControllerEncoding || operator instanceof TupleSourceEncoding - || operator instanceof NChiladaFileScanEncoding || operator instanceof SeaFlowFileScanEncoding + if (operator instanceof CollectConsumerEncoding + || operator instanceof SingletonEncoding + || operator instanceof EOSControllerEncoding + || operator instanceof TupleSourceEncoding + || operator instanceof NChiladaFileScanEncoding + || operator instanceof SeaFlowFileScanEncoding || operator instanceof TipsyFileScanEncoding) { if (fragment.workers == null) { String encodingTypeName = operator.getClass().getSimpleName(); - String operatorTypeName = encodingTypeName.substring(0, encodingTypeName.indexOf("Encoding")); - LOGGER.warn("{} operator can only be instantiated on a single worker, assigning to random worker", + String operatorTypeName = + encodingTypeName.substring(0, encodingTypeName.indexOf("Encoding")); + LOGGER.warn( + "{} operator can only be instantiated on a single worker, assigning to random worker", operatorTypeName); fragment.workers = singletonWorkers; } else { - Preconditions.checkArgument(fragment.workers.size() == 1, - "Fragment %s has a singleton operator %s, but workers %s", fragment.fragmentIndex, operator.opId, + Preconditions.checkArgument( + fragment.workers.size() == 1, + "Fragment %s has a singleton operator %s, but workers %s", + fragment.fragmentIndex, + operator.opId, fragment.workers); } /* We only need to verify singleton-ness once per fragment. */ @@ -371,7 +413,8 @@ private static void fillInRealOperatorAndWorkerIDs(final List || operator instanceof IDBControllerEncoding) { + } else if (operator instanceof AbstractProducerEncoding + || operator instanceof IDBControllerEncoding) { producerWorkerMap.put(operator.opId, fragment.workers); } } @@ -382,15 +425,18 @@ private static void fillInRealOperatorAndWorkerIDs(final List operator : fragment.operators) { if (operator instanceof AbstractConsumerEncoding) { AbstractConsumerEncoding consumer = (AbstractConsumerEncoding) operator; - consumer.setRealWorkerIds(ImmutableSet.copyOf(producerWorkerMap.get(consumer.argOperatorId))); + consumer.setRealWorkerIds( + ImmutableSet.copyOf(producerWorkerMap.get(consumer.argOperatorId))); } else if (operator instanceof AbstractProducerEncoding) { AbstractProducerEncoding producer = (AbstractProducerEncoding) operator; producer.setRealWorkerIds(ImmutableSet.copyOf(consumerWorkerMap.get(producer.opId))); producer.setRealOperatorIds(ImmutableList.copyOf(consumerMap.get(producer.opId))); } else if (operator instanceof IDBControllerEncoding) { IDBControllerEncoding idbController = (IDBControllerEncoding) operator; - idbController.realEosControllerWorkerId = consumerWorkerMap.get(idbController.opId).get(0); - idbController.setRealEosControllerOperatorID(consumerMap.get(idbController.opId).iterator().next()); + idbController.realEosControllerWorkerId = + consumerWorkerMap.get(idbController.opId).get(0); + idbController.setRealEosControllerOperatorID( + consumerMap.get(idbController.opId).iterator().next()); } } } @@ -413,7 +459,8 @@ private static void fillInRealOperatorAndWorkerIDs(final List fragments, final ConstructArgs args) + private static void assignWorkersToFragments( + final List fragments, final ConstructArgs args) throws CatalogException, DbException { /* 1. Honor user overrides. Note this is unchecked, but we may find constraint violations later. */ @@ -461,16 +508,23 @@ private static void assignWorkersToFragments(final List fr * @param args construct args containing the server and query ID. * @param allOperators a map to keep instantiated operators. */ - private static void instantiateFragmentOperators(final PlanFragmentEncoding planFragment, final ConstructArgs args, + private static void instantiateFragmentOperators( + final PlanFragmentEncoding planFragment, + final ConstructArgs args, final Map allOperators) { for (OperatorEncoding encoding : planFragment.operators) { if (allOperators.get(encoding.opId) != null) { - throw new MyriaApiException(Status.BAD_REQUEST, "Multiple operators with opId = " + encoding.opId - + " detected in the fragment: " + planFragment.fragmentIndex); + throw new MyriaApiException( + Status.BAD_REQUEST, + "Multiple operators with opId = " + + encoding.opId + + " detected in the fragment: " + + planFragment.fragmentIndex); } Operator op = encoding.construct(args); /* helpful for debugging. */ - op.setOpName(MoreObjects.firstNonNull(encoding.opName, "Operator" + String.valueOf(encoding.opId))); + op.setOpName( + MoreObjects.firstNonNull(encoding.opName, "Operator" + String.valueOf(encoding.opId))); op.setOpId(encoding.opId); op.setFragmentId(planFragment.fragmentIndex); allOperators.put(encoding.opId, op); @@ -491,21 +545,30 @@ private static void instantiateFragmentOperators(final PlanFragmentEncoding plan * @param allOperators a map to keep instantiated operators. * @return the actual plan fragment. */ - private static RootOperator instantiateFragment(final PlanFragmentEncoding planFragment, final ConstructArgs args, + private static RootOperator instantiateFragment( + final PlanFragmentEncoding planFragment, + final ConstructArgs args, final Map allOperators) { RootOperator fragmentRoot = null; for (OperatorEncoding encoding : planFragment.operators) { Operator op = allOperators.get(encoding.opId); if (op instanceof RootOperator) { if (fragmentRoot != null) { - throw new MyriaApiException(Status.BAD_REQUEST, "Multiple " + RootOperator.class.getSimpleName() - + " detected in the fragment: " + fragmentRoot.getOpName() + ", and " + encoding.opId); + throw new MyriaApiException( + Status.BAD_REQUEST, + "Multiple " + + RootOperator.class.getSimpleName() + + " detected in the fragment: " + + fragmentRoot.getOpName() + + ", and " + + encoding.opId); } fragmentRoot = (RootOperator) op; } } if (fragmentRoot == null) { - throw new MyriaApiException(Status.BAD_REQUEST, + throw new MyriaApiException( + Status.BAD_REQUEST, "No " + RootOperator.class.getSimpleName() + " detected in the fragment."); } return fragmentRoot; @@ -516,8 +579,8 @@ private static RootOperator instantiateFragment(final PlanFragmentEncoding planF * @param allOperators a map to keep instantiated operators. * @return if any more consumer has its schema to be set. */ - private static boolean setConsumerSchema(final List fragments, - final Map allOperators) { + private static boolean setConsumerSchema( + final List fragments, final Map allOperators) { boolean changed = false; for (PlanFragmentEncoding fragment : fragments) { for (OperatorEncoding encoding : fragment.operators) { @@ -526,7 +589,8 @@ private static boolean setConsumerSchema(final List fragme if (consumer.getSchema() != null) { continue; } - Operator producingOp = allOperators.get(((AbstractConsumerEncoding) encoding).argOperatorId); + Operator producingOp = + allOperators.get(((AbstractConsumerEncoding) encoding).argOperatorId); if (producingOp instanceof IDBController) { consumer.setSchema(IDBController.EOI_REPORT_SCHEMA); } else if (producingOp instanceof EOSController) { @@ -552,14 +616,23 @@ private static boolean setConsumerSchema(final List fragme * @param server the server on which the catalog will be updated * @return the query plan to update the master's catalog with the new number of tuples for all written relations. */ - public static SubQuery getRelationTupleUpdateSubQuery(final Map relationsWritten, - final Server server) { + public static SubQuery getRelationTupleUpdateSubQuery( + final Map relationsWritten, final Server server) { ExchangePairID collectId = ExchangePairID.newID(); - Schema schema = Schema.ofFields("userName", Type.STRING_TYPE, "programName", Type.STRING_TYPE, "relationName", - Type.STRING_TYPE, "tupleCount", Type.LONG_TYPE); + Schema schema = + Schema.ofFields( + "userName", + Type.STRING_TYPE, + "programName", + Type.STRING_TYPE, + "relationName", + Type.STRING_TYPE, + "tupleCount", + Type.LONG_TYPE); String dbms = server.getDBMS(); - Preconditions.checkState(dbms != null, "Server must have a configured DBMS environment variable"); + Preconditions.checkState( + dbms != null, "Server must have a configured DBMS environment variable"); /* Worker plans: for each relation, create a {@link DbQueryScan} to get the count, an {@link Apply} to add the * {@link RelationKey}, then a {@link CollectProducer} to send the count to the master. */ @@ -568,15 +641,22 @@ public static SubQuery getRelationTupleUpdateSubQuery(final Map workers = meta.getWorkers(); RelationKey relation = meta.getRelationKey(); for (Integer worker : workers) { - DbQueryScan localCount = new DbQueryScan("SELECT COUNT(*) FROM " + relation.toString(dbms), - Schema.ofFields("tupleCount", Type.LONG_TYPE)); - List expressions = ImmutableList.of( - new Expression(schema.getColumnName(0), new ConstantExpression(relation.getUserName())), - new Expression(schema.getColumnName(1), new ConstantExpression(relation.getProgramName())), - new Expression(schema.getColumnName(2), new ConstantExpression(relation.getRelationName())), - new Expression(schema.getColumnName(3), new VariableExpression(0))); + DbQueryScan localCount = + new DbQueryScan( + "SELECT COUNT(*) FROM " + relation.toString(dbms), + Schema.ofFields("tupleCount", Type.LONG_TYPE)); + List expressions = + ImmutableList.of( + new Expression( + schema.getColumnName(0), new ConstantExpression(relation.getUserName())), + new Expression( + schema.getColumnName(1), new ConstantExpression(relation.getProgramName())), + new Expression( + schema.getColumnName(2), new ConstantExpression(relation.getRelationName())), + new Expression(schema.getColumnName(3), new VariableExpression(0))); Apply addRelationName = new Apply(localCount, expressions); - CollectProducer producer = new CollectProducer(addRelationName, collectId, MyriaConstants.MASTER_ID); + CollectProducer producer = + new CollectProducer(addRelationName, collectId, MyriaConstants.MASTER_ID); if (!workerPlans.containsKey(worker)) { workerPlans.put(worker, new SubQueryPlan(producer)); } else { @@ -587,8 +667,9 @@ public static SubQuery getRelationTupleUpdateSubQuery(final Map mw = null; try { @@ -193,18 +194,22 @@ public void run() { case QUERY_COMPLETE: QueryReport qr = qm.getQueryReport(); if (qr.getSuccess()) { - LOGGER.info("Worker #{} succeeded in executing query #{}.", senderID, subQueryId); + LOGGER.info( + "Worker #{} succeeded in executing query #{}.", senderID, subQueryId); queryManager.workerComplete(subQueryId, senderID); } else { ObjectInputStream osis = null; Throwable cause = null; try { - osis = new ObjectInputStream(new ByteArrayInputStream(qr.getCause().toByteArray())); + osis = + new ObjectInputStream( + new ByteArrayInputStream(qr.getCause().toByteArray())); cause = (Throwable) (osis.readObject()); } catch (IOException | ClassNotFoundException e) { LOGGER.error("Error decoding failure cause", e); } - LOGGER.error("Worker #{} failed in executing query #{}.", senderID, subQueryId, cause); + LOGGER.error( + "Worker #{} failed in executing query #{}.", senderID, subQueryId, cause); queryManager.workerFailed(subQueryId, senderID, cause); } break; @@ -238,7 +243,8 @@ private Optional dequeueDriverMessage() { } private void enqueueDriverMessage(@Nonnull final TransportMessage msg) { - final TaskMessage driverMsg = TaskMessage.from(MyriaConstants.MASTER_ID + "", msg.toByteArray()); + final TaskMessage driverMsg = + TaskMessage.from(MyriaConstants.MASTER_ID + "", msg.toByteArray()); pendingDriverMessages.add(driverMsg); } @@ -272,34 +278,47 @@ public void onNext(final DriverMessage driverMessage) { workerLock.lock(); try { switch (controlM.getType()) { - case REMOVE_WORKER: { - LOGGER.info("Driver reported worker {} as dead, removing from alive workers.", workerId); - aliveWorkers.remove(workerId); - queryManager.workerDied(workerId); - connectionPool.removeRemote(workerId).addListener(new ChannelGroupFutureListener() { - @Override - public void operationComplete(final ChannelGroupFuture future) { - if (future.isCompleteSuccess()) { - LOGGER.info("removed connection for remote worker {} from connection pool", workerId); - } else { - LOGGER.info("failed to remove connection for remote worker {} from connection pool", workerId); - } - } - }); - enqueueDriverMessage(IPCUtils.removeWorkerAckTM(workerId)); - } + case REMOVE_WORKER: + { + LOGGER.info( + "Driver reported worker {} as dead, removing from alive workers.", workerId); + aliveWorkers.remove(workerId); + queryManager.workerDied(workerId); + connectionPool + .removeRemote(workerId) + .addListener( + new ChannelGroupFutureListener() { + @Override + public void operationComplete(final ChannelGroupFuture future) { + if (future.isCompleteSuccess()) { + LOGGER.info( + "removed connection for remote worker {} from connection pool", + workerId); + } else { + LOGGER.info( + "failed to remove connection for remote worker {} from connection pool", + workerId); + } + } + }); + enqueueDriverMessage(IPCUtils.removeWorkerAckTM(workerId)); + } break; - case ADD_WORKER: { - Preconditions.checkState(!aliveWorkers.contains(workerId)); - LOGGER.info("Driver wants to add worker {} to alive workers.", workerId); - connectionPool.putRemote(workerId, SocketInfo.fromProtobuf(controlM.getRemoteAddress())); - queryManager.workerRestarted(workerId, ImmutableSet.copyOf(controlM.getAckedWorkerIdsList())); - aliveWorkers.add(workerId); - enqueueDriverMessage(IPCUtils.addWorkerAckTM(workerId)); - } + case ADD_WORKER: + { + Preconditions.checkState(!aliveWorkers.contains(workerId)); + LOGGER.info("Driver wants to add worker {} to alive workers.", workerId); + connectionPool.putRemote( + workerId, SocketInfo.fromProtobuf(controlM.getRemoteAddress())); + queryManager.workerRestarted( + workerId, ImmutableSet.copyOf(controlM.getAckedWorkerIdsList())); + aliveWorkers.add(workerId); + enqueueDriverMessage(IPCUtils.addWorkerAckTM(workerId)); + } break; default: - throw new IllegalStateException("Unexpected driver control message type: " + controlM.getType()); + throw new IllegalStateException( + "Unexpected driver control message type: " + controlM.getType()); } } finally { workerLock.unlock(); @@ -418,7 +437,8 @@ QueryExecutionMode getExecutionMode() { * @param injector a Tang injector for instantiating objects from configuration */ @Inject - public Server(@Parameter(MasterHost.class) final String masterHost, + public Server( + @Parameter(MasterHost.class) final String masterHost, @Parameter(MasterRpcPort.class) final int masterPort, @Parameter(DefaultInstancePath.class) final String catalogPath, @Parameter(StorageDbms.class) final String databaseSystem, @@ -429,7 +449,8 @@ public Server(@Parameter(MasterHost.class) final String masterHost, @Parameter(FlowControlWriteBufferHighMarkBytes.class) final int writeBufferHighWaterMark, @Parameter(OperatorInputBufferCapacity.class) final int inputBufferCapacity, @Parameter(OperatorInputBufferRecoverTrigger.class) final int inputBufferRecoverTrigger, - @Parameter(PersistUri.class) final String persistURI, final Injector injector) { + @Parameter(PersistUri.class) final String persistURI, + final Injector injector) { this.connectTimeoutMillis = connectTimeoutMillis; this.sendBufferSize = sendBufferSize; @@ -480,17 +501,20 @@ private ImmutableSet getWorkerConfs(final Injector injector) return workerConfsBuilder.build(); } - private static Integer getIdFromWorkerConf(final Configuration workerConf) throws InjectionException { + private static Integer getIdFromWorkerConf(final Configuration workerConf) + throws InjectionException { final Injector injector = Tang.Factory.getTang().newInjector(workerConf); return injector.getNamedInstance(MyriaWorkerConfigurationModule.WorkerId.class); } - private static String getHostFromWorkerConf(final Configuration workerConf) throws InjectionException { + private static String getHostFromWorkerConf(final Configuration workerConf) + throws InjectionException { final Injector injector = Tang.Factory.getTang().newInjector(workerConf); return injector.getNamedInstance(MyriaWorkerConfigurationModule.WorkerHost.class); } - private static Integer getPortFromWorkerConf(final Configuration workerConf) throws InjectionException { + private static Integer getPortFromWorkerConf(final Configuration workerConf) + throws InjectionException { final Injector injector = Tang.Factory.getTang().newInjector(workerConf); return injector.getNamedInstance(MyriaWorkerConfigurationModule.WorkerPort.class); } @@ -538,7 +562,8 @@ public void start() throws Exception { final ImmutableSet workerConfs = getWorkerConfs(injector); final ImmutableMap.Builder workersBuilder = ImmutableMap.builder(); for (Configuration workerConf : workerConfs) { - workersBuilder.put(getIdFromWorkerConf(workerConf), + workersBuilder.put( + getIdFromWorkerConf(workerConf), new SocketInfo(getHostFromWorkerConf(workerConf), getPortFromWorkerConf(workerConf))); } workers = workersBuilder.build(); @@ -552,31 +577,51 @@ public void start() throws Exception { LOGGER.info("Attempting to open master catalog file under {}...", catalogPath); catalog = MasterCatalog.open(catalogPath); } catch (FileNotFoundException e) { - LOGGER.info("Failed to open master catalog file under {}, attempting to create it...\n({})", catalogPath, + LOGGER.info( + "Failed to open master catalog file under {}, attempting to create it...\n({})", + catalogPath, e.getMessage()); catalog = MasterCatalog.create(catalogPath); } queryManager = new QueryManager(catalog, this); - connectionPool = new IPCConnectionPool(MyriaConstants.MASTER_ID, computingUnits, - IPCConfigurations.createMasterIPCServerBootstrap(connectTimeoutMillis, sendBufferSize, receiveBufferSize, - writeBufferLowWaterMark, writeBufferHighWaterMark), - IPCConfigurations.createMasterIPCClientBootstrap(connectTimeoutMillis, sendBufferSize, receiveBufferSize, - writeBufferLowWaterMark, writeBufferHighWaterMark), - new TransportMessageSerializer(), new QueueBasedShortMessageProcessor(messageQueue), - inputBufferCapacity, inputBufferRecoverTrigger); - - scheduledTaskExecutor = Executors - .newSingleThreadScheduledExecutor(new RenamingThreadFactory("Master global timer")); - scheduledTaskExecutor.scheduleAtFixedRate(new DebugHelper(), DebugHelper.INTERVAL, DebugHelper.INTERVAL, - TimeUnit.MILLISECONDS); - messageProcessingExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master message processor")); - serverQueryExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master query executor")); + connectionPool = + new IPCConnectionPool( + MyriaConstants.MASTER_ID, + computingUnits, + IPCConfigurations.createMasterIPCServerBootstrap( + connectTimeoutMillis, + sendBufferSize, + receiveBufferSize, + writeBufferLowWaterMark, + writeBufferHighWaterMark), + IPCConfigurations.createMasterIPCClientBootstrap( + connectTimeoutMillis, + sendBufferSize, + receiveBufferSize, + writeBufferLowWaterMark, + writeBufferHighWaterMark), + new TransportMessageSerializer(), + new QueueBasedShortMessageProcessor(messageQueue), + inputBufferCapacity, + inputBufferRecoverTrigger); + + scheduledTaskExecutor = + Executors.newSingleThreadScheduledExecutor( + new RenamingThreadFactory("Master global timer")); + scheduledTaskExecutor.scheduleAtFixedRate( + new DebugHelper(), DebugHelper.INTERVAL, DebugHelper.INTERVAL, TimeUnit.MILLISECONDS); + messageProcessingExecutor = + Executors.newCachedThreadPool(new RenamingThreadFactory("Master message processor")); + serverQueryExecutor = + Executors.newCachedThreadPool(new RenamingThreadFactory("Master query executor")); /** The {@link Executor} who deals with IPC connection setup/cleanup. */ - ExecutorService ipcBossExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master IPC boss")); + ExecutorService ipcBossExecutor = + Executors.newCachedThreadPool(new RenamingThreadFactory("Master IPC boss")); /** The {@link Executor} who deals with IPC message delivering and transformation. */ - ExecutorService ipcWorkerExecutor = Executors.newCachedThreadPool(new RenamingThreadFactory("Master IPC worker")); + ExecutorService ipcWorkerExecutor = + Executors.newCachedThreadPool(new RenamingThreadFactory("Master IPC worker")); ipcPipelineExecutor = null; // Remove the pipeline executor. // new @@ -588,35 +633,51 @@ public void start() throws Exception { // new RenamingThreadFactory("Master Pipeline executor")); /** The {@link ChannelFactory} for creating client side connections. */ - ChannelFactory clientChannelFactory = new NioClientSocketChannelFactory(ipcBossExecutor, ipcWorkerExecutor, - Runtime.getRuntime().availableProcessors() * 2 + 1); + ChannelFactory clientChannelFactory = + new NioClientSocketChannelFactory( + ipcBossExecutor, ipcWorkerExecutor, Runtime.getRuntime().availableProcessors() * 2 + 1); /** The {@link ChannelFactory} for creating server side accepted connections. */ - ChannelFactory serverChannelFactory = new NioServerSocketChannelFactory(ipcBossExecutor, ipcWorkerExecutor, - Runtime.getRuntime().availableProcessors() * 2 + 1); + ChannelFactory serverChannelFactory = + new NioServerSocketChannelFactory( + ipcBossExecutor, ipcWorkerExecutor, Runtime.getRuntime().availableProcessors() * 2 + 1); // Start server with Nb of active threads = 2*NB CPU + 1 as maximum. - ChannelPipelineFactory serverPipelineFactory = new IPCPipelineFactories.MasterServerPipelineFactory(connectionPool, - getPipelineExecutor()); - ChannelPipelineFactory clientPipelineFactory = new IPCPipelineFactories.MasterClientPipelineFactory(connectionPool, - getPipelineExecutor()); - ChannelPipelineFactory masterInJVMPipelineFactory = new IPCPipelineFactories.MasterInJVMPipelineFactory( - connectionPool); - - connectionPool.start(serverChannelFactory, serverPipelineFactory, clientChannelFactory, clientPipelineFactory, - masterInJVMPipelineFactory, new InJVMLoopbackChannelSink()); + ChannelPipelineFactory serverPipelineFactory = + new IPCPipelineFactories.MasterServerPipelineFactory(connectionPool, getPipelineExecutor()); + ChannelPipelineFactory clientPipelineFactory = + new IPCPipelineFactories.MasterClientPipelineFactory(connectionPool, getPipelineExecutor()); + ChannelPipelineFactory masterInJVMPipelineFactory = + new IPCPipelineFactories.MasterInJVMPipelineFactory(connectionPool); + + connectionPool.start( + serverChannelFactory, + serverPipelineFactory, + clientChannelFactory, + clientPipelineFactory, + masterInJVMPipelineFactory, + new InJVMLoopbackChannelSink()); messageProcessingExecutor.submit(new MessageProcessor()); LOGGER.info("Server started on {}", masterSocketInfo); if (getDBMS().equals(MyriaConstants.STORAGE_SYSTEM_POSTGRESQL)) { final List workerIds = ImmutableList.copyOf(workers.keySet()); - addRelationToCatalog(MyriaConstants.EVENT_PROFILING_RELATION, MyriaConstants.EVENT_PROFILING_SCHEMA, workerIds, + addRelationToCatalog( + MyriaConstants.EVENT_PROFILING_RELATION, + MyriaConstants.EVENT_PROFILING_SCHEMA, + workerIds, false); - addRelationToCatalog(MyriaConstants.SENT_PROFILING_RELATION, MyriaConstants.SENT_PROFILING_SCHEMA, workerIds, + addRelationToCatalog( + MyriaConstants.SENT_PROFILING_RELATION, + MyriaConstants.SENT_PROFILING_SCHEMA, + workerIds, + false); + addRelationToCatalog( + MyriaConstants.RESOURCE_PROFILING_RELATION, + MyriaConstants.RESOURCE_PROFILING_SCHEMA, + workerIds, false); - addRelationToCatalog(MyriaConstants.RESOURCE_PROFILING_RELATION, MyriaConstants.RESOURCE_PROFILING_SCHEMA, - workerIds, false); } } @@ -629,8 +690,12 @@ public void start() throws Exception { * @param force force add the relation; will replace an existing entry. * @throws DbException if the catalog cannot be accessed */ - private void addRelationToCatalog(final RelationKey relationKey, final Schema schema, final List workers, - final boolean force) throws DbException { + private void addRelationToCatalog( + final RelationKey relationKey, + final Schema schema, + final List workers, + final boolean force) + throws DbException { try { if (!force && getSchema(relationKey) != null) { return; @@ -643,8 +708,14 @@ private void addRelationToCatalog(final RelationKey relationKey, final Schema sc long queryId = catalog.newQuery(query); - final Query queryState = new Query(queryId, query, - new SubQuery(new SubQueryPlan(new EmptySink(new EOSSource())), new HashMap()), this); + final Query queryState = + new Query( + queryId, + query, + new SubQuery( + new SubQueryPlan(new EmptySink(new EOSSource())), + new HashMap()), + this); queryState.markSuccess(); catalog.queryFinished(queryState); @@ -675,15 +746,20 @@ public String getDBMS() { * @throws DbException if any error occurs. * @throws CatalogException catalog errors. */ - public QueryFuture submitQueryPlan(final RootOperator masterRoot, final Map workerRoots) + public QueryFuture submitQueryPlan( + final RootOperator masterRoot, final Map workerRoots) throws DbException, CatalogException { String catalogInfoPlaceHolder = "MasterPlan: " + masterRoot + "; WorkerPlan: " + workerRoots; Map workerPlans = new HashMap<>(); for (Entry entry : workerRoots.entrySet()) { workerPlans.put(entry.getKey(), new SubQueryPlan(entry.getValue())); } - return queryManager.submitQuery(catalogInfoPlaceHolder, catalogInfoPlaceHolder, catalogInfoPlaceHolder, - new SubQueryPlan(masterRoot), workerPlans); + return queryManager.submitQuery( + catalogInfoPlaceHolder, + catalogInfoPlaceHolder, + catalogInfoPlaceHolder, + new SubQueryPlan(masterRoot), + workerPlans); } /** @return the set of workers that are currently alive. */ @@ -698,7 +774,8 @@ public Set getAliveWorkers() { * @return a subset of workers that are currently alive. */ public Set getRandomWorkers(final int number) { - Preconditions.checkArgument(number <= getAliveWorkers().size(), + Preconditions.checkArgument( + number <= getAliveWorkers().size(), "The number of workers requested cannot exceed the number of alive workers."); if (number == getAliveWorkers().size()) { return getAliveWorkers(); @@ -725,8 +802,12 @@ public Map getWorkers() { * @throws InterruptedException interrupted * @throws DbException if there is an error */ - public DatasetStatus ingestDataset(final RelationKey relationKey, List workersToIngest, - final List> indexes, final Operator source, final DistributeFunction df) + public DatasetStatus ingestDataset( + final RelationKey relationKey, + List workersToIngest, + final List> indexes, + final Operator source, + final DistributeFunction df) throws InterruptedException, DbException { /* Figure out the workers we will use. If workersToIngest is null, use all active workers. */ if (workersToIngest == null) { @@ -738,11 +819,12 @@ public DatasetStatus ingestDataset(final RelationKey relationKey, List /* The master plan: send the tuples out. */ ExchangePairID scatterId = ExchangePairID.newID(); df.setNumDestinations(workersArray.length, 1); - GenericShuffleProducer scatter = new GenericShuffleProducer(source, new ExchangePairID[] { scatterId }, - workersArray, df); + GenericShuffleProducer scatter = + new GenericShuffleProducer(source, new ExchangePairID[] {scatterId}, workersArray, df); /* The workers' plan */ - Consumer gather = new Consumer(source.getSchema(), scatterId, ImmutableSet.of(MyriaConstants.MASTER_ID)); + Consumer gather = + new Consumer(source.getSchema(), scatterId, ImmutableSet.of(MyriaConstants.MASTER_ID)); DbInsert insert = new DbInsert(gather, relationKey, true, indexes); Map workerPlans = new HashMap<>(); for (Integer workerId : workersArray) { @@ -751,8 +833,13 @@ public DatasetStatus ingestDataset(final RelationKey relationKey, List ListenableFuture qf; try { - qf = queryManager.submitQuery("ingest " + relationKey.toString(), "ingest " + relationKey.toString(), - "ingest " + relationKey.toString(getDBMS()), new SubQueryPlan(scatter), workerPlans); + qf = + queryManager.submitQuery( + "ingest " + relationKey.toString(), + "ingest " + relationKey.toString(), + "ingest " + relationKey.toString(getDBMS()), + new SubQueryPlan(scatter), + workerPlans); } catch (CatalogException e) { throw new DbException("Error submitting query", e); } @@ -776,10 +863,17 @@ public DatasetStatus ingestDataset(final RelationKey relationKey, List * @throws DbException * @throws InterruptedException */ - public DatasetStatus parallelIngestDataset(final RelationKey relationKey, final Schema schema, - @Nullable final Character delimiter, @Nullable final Character quote, @Nullable final Character escape, - @Nullable final Integer numberOfSkippedLines, final AmazonS3Source s3Source, final Set workersToIngest, - final DistributeFunction distributeFunction) throws URIException, DbException, InterruptedException { + public DatasetStatus parallelIngestDataset( + final RelationKey relationKey, + final Schema schema, + @Nullable final Character delimiter, + @Nullable final Character quote, + @Nullable final Character escape, + @Nullable final Integer numberOfSkippedLines, + final AmazonS3Source s3Source, + final Set workersToIngest, + final DistributeFunction distributeFunction) + throws URIException, DbException, InterruptedException { /* Figure out the workers we will use */ Set actualWorkers = workersToIngest; long fileSize = s3Source.getFileSize(); @@ -817,15 +911,31 @@ public DatasetStatus parallelIngestDataset(final RelationKey relationKey, final endRange = (partitionSize * workerID) - 1; } - CSVFileScanFragment scanFragment = new CSVFileScanFragment(s3Source, schema, startRange, endRange, isLastWorker, - delimiter, quote, escape, numberOfSkippedLines); - workerPlans.put(workersArray[workerID - 1], new SubQueryPlan(new DbInsert(scanFragment, relationKey, true))); + CSVFileScanFragment scanFragment = + new CSVFileScanFragment( + s3Source, + schema, + startRange, + endRange, + isLastWorker, + delimiter, + quote, + escape, + numberOfSkippedLines); + workerPlans.put( + workersArray[workerID - 1], + new SubQueryPlan(new DbInsert(scanFragment, relationKey, true))); } ListenableFuture qf; try { - qf = queryManager.submitQuery("ingest " + relationKey.toString(), "ingest " + relationKey.toString(), - "ingest " + relationKey.toString(getDBMS()), new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans); + qf = + queryManager.submitQuery( + "ingest " + relationKey.toString(), + "ingest " + relationKey.toString(), + "ingest " + relationKey.toString(getDBMS()), + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans); } catch (CatalogException e) { throw new DbException("Error submitting query", e); } @@ -847,8 +957,9 @@ public DatasetStatus parallelIngestDataset(final RelationKey relationKey, final * @throws DbException if there is an error * @throws InterruptedException interrupted */ - public void addDatasetToCatalog(final RelationKey relationKey, final Schema schema, - final List workersToImportFrom) throws DbException, InterruptedException { + public void addDatasetToCatalog( + final RelationKey relationKey, final Schema schema, final List workersToImportFrom) + throws DbException, InterruptedException { /* Figure out the workers we will use. If workersToImportFrom is null, use all active workers. */ List actualWorkers = workersToImportFrom; @@ -860,11 +971,16 @@ public void addDatasetToCatalog(final RelationKey relationKey, final Schema sche try { Map workerPlans = new HashMap<>(); for (Integer workerId : actualWorkers) { - workerPlans.put(workerId, new SubQueryPlan(new DbInsert(EmptyRelation.of(schema), relationKey, false))); + workerPlans.put( + workerId, new SubQueryPlan(new DbInsert(EmptyRelation.of(schema), relationKey, false))); } - ListenableFuture qf = queryManager.submitQuery("add to catalog " + relationKey.toString(), - "add to catalog " + relationKey.toString(), "add to catalog " + relationKey.toString(getDBMS()), - new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans); + ListenableFuture qf = + queryManager.submitQuery( + "add to catalog " + relationKey.toString(), + "add to catalog " + relationKey.toString(), + "add to catalog " + relationKey.toString(getDBMS()), + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans); try { qf.get(); } catch (ExecutionException e) { @@ -881,7 +997,8 @@ public void addDatasetToCatalog(final RelationKey relationKey, final Schema sche * @throws DbException if there is an error * @throws InterruptedException interrupted */ - public void deleteDataset(final RelationKey relationKey) throws DbException, InterruptedException { + public void deleteDataset(final RelationKey relationKey) + throws DbException, InterruptedException { /* Mark the relation as is_deleted */ try { @@ -894,12 +1011,18 @@ public void deleteDataset(final RelationKey relationKey) throws DbException, Int try { Map workerPlans = new HashMap<>(); for (Integer workerId : getWorkersForRelation(relationKey, null)) { - workerPlans.put(workerId, - new SubQueryPlan(new DbDelete(EmptyRelation.of(catalog.getSchema(relationKey)), relationKey, null))); + workerPlans.put( + workerId, + new SubQueryPlan( + new DbDelete(EmptyRelation.of(catalog.getSchema(relationKey)), relationKey, null))); } - ListenableFuture qf = queryManager.submitQuery("delete " + relationKey.toString(), - "delete " + relationKey.toString(), "deleting from " + relationKey.toString(getDBMS()), - new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans); + ListenableFuture qf = + queryManager.submitQuery( + "delete " + relationKey.toString(), + "delete " + relationKey.toString(), + "deleting from " + relationKey.toString(getDBMS()), + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans); try { qf.get(); } catch (ExecutionException e) { @@ -918,19 +1041,31 @@ public void deleteDataset(final RelationKey relationKey) throws DbException, Int } /** Create indexes and add the metadata to the catalog */ - public long addIndexesToRelation(final RelationKey relationKey, final Schema schema, final List indexes) + public long addIndexesToRelation( + final RelationKey relationKey, final Schema schema, final List indexes) throws DbException, InterruptedException { long queryID; /* Add indexes to relations */ try { Map workerPlans = new HashMap<>(); for (Integer workerId : getWorkersForRelation(relationKey, null)) { - workerPlans.put(workerId, new SubQueryPlan( - new DbCreateIndex(EmptyRelation.of(catalog.getSchema(relationKey)), relationKey, schema, indexes, null))); + workerPlans.put( + workerId, + new SubQueryPlan( + new DbCreateIndex( + EmptyRelation.of(catalog.getSchema(relationKey)), + relationKey, + schema, + indexes, + null))); } - ListenableFuture qf = queryManager.submitQuery("add indexes to " + relationKey.toString(), - "add indexes to " + relationKey.toString(), "add indexes to " + relationKey.toString(getDBMS()), - new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans); + ListenableFuture qf = + queryManager.submitQuery( + "add indexes to " + relationKey.toString(), + "add indexes to " + relationKey.toString(), + "add indexes to " + relationKey.toString(getDBMS()), + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans); try { queryID = qf.get().getQueryId(); } catch (ExecutionException e) { @@ -951,7 +1086,8 @@ public long addIndexesToRelation(final RelationKey relationKey, final Schema sch } /** Create a view */ - public long createView(final String viewName, final String viewDefinition, final Set workers) + public long createView( + final String viewName, final String viewDefinition, final Set workers) throws DbException, InterruptedException { long queryID; Set actualWorkers = workers; @@ -963,11 +1099,19 @@ public long createView(final String viewName, final String viewDefinition, final try { Map workerPlans = new HashMap<>(); for (Integer workerId : actualWorkers) { - workerPlans.put(workerId, - new SubQueryPlan(new DbCreateView(EmptyRelation.of(Schema.EMPTY_SCHEMA), viewName, viewDefinition, null))); + workerPlans.put( + workerId, + new SubQueryPlan( + new DbCreateView( + EmptyRelation.of(Schema.EMPTY_SCHEMA), viewName, viewDefinition, null))); } - ListenableFuture qf = queryManager.submitQuery("create view", "create view", "create view", - new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans); + ListenableFuture qf = + queryManager.submitQuery( + "create view", + "create view", + "create view", + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans); try { queryID = qf.get().getQueryId(); } catch (ExecutionException e) { @@ -988,8 +1132,12 @@ public long createView(final String viewName, final String viewDefinition, final * @param outputSchema the output schema of the function * @return the status of the function */ - public String createFunction(final String name, final String definition, final String outputSchema, - final Set workers) throws DbException, InterruptedException { + public String createFunction( + final String name, + final String definition, + final String outputSchema, + final Set workers) + throws DbException, InterruptedException { String response = "Created Function"; Set actualWorkers = workers; if (workers == null) { @@ -1002,17 +1150,26 @@ public String createFunction(final String name, final String definition, final S if (matcher.matches()) { /* Add a replace statement */ - String modifiedReplaceFunction = definition.replace("CREATE FUNCTION", "CREATE OR REPLACE FUNCTION"); + String modifiedReplaceFunction = + definition.replace("CREATE FUNCTION", "CREATE OR REPLACE FUNCTION"); /* Create the function */ try { Map workerPlans = new HashMap<>(); for (Integer workerId : actualWorkers) { - workerPlans.put(workerId, - new SubQueryPlan(new DbExecute(EmptyRelation.of(Schema.EMPTY_SCHEMA), modifiedReplaceFunction, null))); + workerPlans.put( + workerId, + new SubQueryPlan( + new DbExecute( + EmptyRelation.of(Schema.EMPTY_SCHEMA), modifiedReplaceFunction, null))); } - ListenableFuture qf = queryManager.submitQuery("create function", "create function", "create function", - new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans); + ListenableFuture qf = + queryManager.submitQuery( + "create function", + "create function", + "create function", + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans); try { qf.get().getQueryId(); } catch (ExecutionException e) { @@ -1053,17 +1210,32 @@ public long persistDataset(final RelationKey relationKey) /* Create the query plan for persist */ try { - ImmutableMap.Builder workerPlans = new ImmutableMap.Builder(); + ImmutableMap.Builder workerPlans = + new ImmutableMap.Builder(); for (Integer workerId : getWorkersForRelation(relationKey, null)) { - String partitionName = String.format(persistURI + "/myria-system/partition-%s/%s/%s/%s", workerId, - relationKey.getUserName(), relationKey.getProgramName(), relationKey.getRelationName()); + String partitionName = + String.format( + persistURI + "/myria-system/partition-%s/%s/%s/%s", + workerId, + relationKey.getUserName(), + relationKey.getProgramName(), + relationKey.getRelationName()); DataSink workerSink = new UriSink(partitionName); - workerPlans.put(workerId, new SubQueryPlan(new TupleSink(new DbQueryScan(relationKey, getSchema(relationKey)), - new PostgresBinaryTupleWriter(), workerSink))); + workerPlans.put( + workerId, + new SubQueryPlan( + new TupleSink( + new DbQueryScan(relationKey, getSchema(relationKey)), + new PostgresBinaryTupleWriter(), + workerSink))); } - ListenableFuture qf = queryManager.submitQuery("persist " + relationKey.toString(), - "persist " + relationKey.toString(), "persisting from " + relationKey.toString(getDBMS()), - new SubQueryPlan(new EmptySink(new EOSSource())), workerPlans.build()); + ListenableFuture qf = + queryManager.submitQuery( + "persist " + relationKey.toString(), + "persist " + relationKey.toString(), + "persisting from " + relationKey.toString(getDBMS()), + new SubQueryPlan(new EmptySink(new EOSSource())), + workerPlans.build()); try { queryID = qf.get().getQueryId(); } catch (ExecutionException e) { @@ -1088,7 +1260,8 @@ public Schema getSchema(final RelationKey relationKey) throws CatalogException { * @param key the relation key. * @param howPartitioned how the dataset was partitioned. */ - public void updateHowDistributed(final RelationKey key, final HowDistributed howDistributed) throws DbException { + public void updateHowDistributed(final RelationKey key, final HowDistributed howDistributed) + throws DbException { try { catalog.updateHowDistributed(key, howDistributed); } catch (CatalogException e) { @@ -1102,8 +1275,8 @@ public void updateHowDistributed(final RelationKey key, final HowDistributed how * @return the list of workers that store the specified relation. * @throws CatalogException if there is an error accessing the catalog. */ - public Set getWorkersForRelation(final RelationKey relationKey, final Integer storedRelationId) - throws CatalogException { + public Set getWorkersForRelation( + final RelationKey relationKey, final Integer storedRelationId) throws CatalogException { return catalog.getWorkersForRelation(relationKey, storedRelationId); } @@ -1112,7 +1285,8 @@ public Set getWorkersForRelation(final RelationKey relationKey, final I * @param relationKey the key of the desired temp relation. * @return the list of workers that store the specified relation. */ - public Set getWorkersForTempRelation(@Nonnull final Long queryId, @Nonnull final RelationKey relationKey) { + public Set getWorkersForTempRelation( + @Nonnull final Long queryId, @Nonnull final RelationKey relationKey) { return queryManager.getQuery(queryId).getWorkersForTempRelation(relationKey); } @@ -1180,7 +1354,8 @@ public List getDatasetsForUser(final String userName) throws DbEx * @return a list of datasets belonging to the specified program. * @throws DbException if there is an error accessing the Catalog. */ - public List getDatasetsForProgram(final String userName, final String programName) throws DbException { + public List getDatasetsForProgram(final String userName, final String programName) + throws DbException { try { return catalog.getDatasetsForProgram(userName, programName); } catch (CatalogException e) { @@ -1228,8 +1403,9 @@ public long getMinQuery(final String searchTerm) throws CatalogException { * @return the query future from which the query status can be looked up. * @throws DbException if there is an error in the system. */ - public ListenableFuture startDataStream(final RelationKey relationKey, final TupleWriter writer, - final DataSink dataSink) throws DbException { + public ListenableFuture startDataStream( + final RelationKey relationKey, final TupleWriter writer, final DataSink dataSink) + throws DbException { /* Get the relation's schema, to make sure it exists. */ final Schema schema; try { @@ -1287,11 +1463,12 @@ public ListenableFuture startDataStream(final RelationKey relationKey, fi * @return the query future from which the query status can be looked up. * @throws DbException if there is an error in the system. */ - public ListenableFuture startTestDataStream(final int numTB, final TupleWriter writer, final DataSink dataSink) - throws DbException { + public ListenableFuture startTestDataStream( + final int numTB, final TupleWriter writer, final DataSink dataSink) throws DbException { - final Schema schema = new Schema(ImmutableList.of(Type.LONG_TYPE, Type.STRING_TYPE), - ImmutableList.of("id", "name")); + final Schema schema = + new Schema( + ImmutableList.of(Type.LONG_TYPE, Type.STRING_TYPE), ImmutableList.of("id", "name")); Random r = new Random(); final TupleBatchBuffer tbb = new TupleBatchBuffer(schema); @@ -1339,8 +1516,12 @@ public ListenableFuture startTestDataStream(final int numTB, final TupleW * @return profiling logs for the query. * @throws DbException if there is an error when accessing profiling logs. */ - public ListenableFuture startSentLogDataStream(final SubQueryId subqueryId, final long fragmentId, - final TupleWriter writer, final DataSink dataSink) throws DbException { + public ListenableFuture startSentLogDataStream( + final SubQueryId subqueryId, + final long fragmentId, + final TupleWriter writer, + final DataSink dataSink) + throws DbException { Set actualWorkers = getWorkersForSubQuery(subqueryId); String fragmentWhere = ""; @@ -1348,13 +1529,21 @@ public ListenableFuture startSentLogDataStream(final SubQueryId subqueryI fragmentWhere = "AND \"fragmentId\" = " + fragmentId; } - final Schema schema = Schema.ofFields("fragmentId", Type.INT_TYPE, "destWorker", Type.INT_TYPE, "numTuples", - Type.LONG_TYPE); + final Schema schema = + Schema.ofFields( + "fragmentId", Type.INT_TYPE, "destWorker", Type.INT_TYPE, "numTuples", Type.LONG_TYPE); - String sentQueryString = Joiner.on(' ').join( - "SELECT \"fragmentId\", \"destWorkerId\", sum(\"numTuples\") as \"numTuples\" FROM", - MyriaConstants.SENT_PROFILING_RELATION.toString(getDBMS()), "WHERE \"queryId\" =", subqueryId.getQueryId(), - "AND \"subQueryId\" =", subqueryId.getSubqueryId(), fragmentWhere, "GROUP BY \"fragmentId\", \"destWorkerId\""); + String sentQueryString = + Joiner.on(' ') + .join( + "SELECT \"fragmentId\", \"destWorkerId\", sum(\"numTuples\") as \"numTuples\" FROM", + MyriaConstants.SENT_PROFILING_RELATION.toString(getDBMS()), + "WHERE \"queryId\" =", + subqueryId.getQueryId(), + "AND \"subQueryId\" =", + subqueryId.getSubqueryId(), + fragmentWhere, + "GROUP BY \"fragmentId\", \"destWorkerId\""); DbQueryScan scan = new DbQueryScan(sentQueryString, schema); final ExchangePairID operatorId = ExchangePairID.newID(); @@ -1370,7 +1559,8 @@ public ListenableFuture startSentLogDataStream(final SubQueryId subqueryI Apply addWorkerId = new Apply(scan, emitExpressions.build()); - CollectProducer producer = new CollectProducer(addWorkerId, operatorId, MyriaConstants.MASTER_ID); + CollectProducer producer = + new CollectProducer(addWorkerId, operatorId, MyriaConstants.MASTER_ID); SubQueryPlan workerPlan = new SubQueryPlan(producer); Map workerPlans = new HashMap<>(actualWorkers.size()); @@ -1378,10 +1568,12 @@ public ListenableFuture startSentLogDataStream(final SubQueryId subqueryI workerPlans.put(worker, workerPlan); } - final Consumer consumer = new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); - final MultiGroupByAggregate aggregate = new MultiGroupByAggregate(consumer, new int[] { 0, 1, 2 }, - new SingleColumnAggregatorFactory(3, AggregationOp.SUM)); + final MultiGroupByAggregate aggregate = + new MultiGroupByAggregate( + consumer, new int[] {0, 1, 2}, new SingleColumnAggregatorFactory(3, AggregationOp.SUM)); // rename columns ImmutableList.Builder renameExpressions = ImmutableList.builder(); @@ -1395,8 +1587,16 @@ public ListenableFuture startSentLogDataStream(final SubQueryId subqueryI final SubQueryPlan masterPlan = new SubQueryPlan(output); /* Submit the plan for the download. */ - String planString = Joiner.on("").join("download profiling sent data for (query=", subqueryId.getQueryId(), - ", subquery=", subqueryId.getSubqueryId(), ", fragment=", fragmentId, ")"); + String planString = + Joiner.on("") + .join( + "download profiling sent data for (query=", + subqueryId.getQueryId(), + ", subquery=", + subqueryId.getSubqueryId(), + ", fragment=", + fragmentId, + ")"); try { return queryManager.submitQuery(planString, planString, planString, masterPlan, workerPlans); } catch (CatalogException e) { @@ -1419,14 +1619,15 @@ private Set getWorkersFromSubqueryPlan(final String plan) { List> fragments; Set actualWorkers = Sets.newHashSet(); try { - fragments = mapper.readValue(plan, new TypeReference>>() { - }); + fragments = mapper.readValue(plan, new TypeReference>>() {}); int fragIdx = 0; for (Map m : fragments) { Object fragWorkers = m.get("workers"); Preconditions.checkNotNull(fragWorkers, "No workers recorded for fragment %s", fragIdx); - Preconditions.checkState(fragWorkers instanceof Collection, - "Expected fragWorkers to be a collection, instead found %s", fragWorkers.getClass()); + Preconditions.checkState( + fragWorkers instanceof Collection, + "Expected fragWorkers to be a collection, instead found %s", + fragWorkers.getClass()); try { @SuppressWarnings("unchecked") Collection curWorkers = (Collection) fragWorkers; @@ -1437,7 +1638,8 @@ private Set getWorkersFromSubqueryPlan(final String plan) { } } } catch (IOException e) { - throw new IllegalArgumentException("Error deserializing workers from encoded plan " + plan, e); + throw new IllegalArgumentException( + "Error deserializing workers from encoded plan " + plan, e); } /* Remove the MASTER from the set. */ actualWorkers.remove(MyriaConstants.MASTER_ID); @@ -1458,7 +1660,8 @@ private Set getWorkersForSubQuery(final SubQueryId subQueryId) throws D } catch (CatalogException e) { throw new DbException(e); } - Preconditions.checkArgument(serializedPlan != null, "No cached query plan for subquery %s", subQueryId); + Preconditions.checkArgument( + serializedPlan != null, "No cached query plan for subquery %s", subQueryId); return getWorkersFromSubqueryPlan(serializedPlan); } @@ -1469,17 +1672,32 @@ private Set getWorkersForSubQuery(final SubQueryId subQueryId) throws D * @return profiling logs for the query. * @throws DbException if there is an error when accessing profiling logs. */ - public ListenableFuture startAggregatedSentLogDataStream(final SubQueryId subqueryId, final TupleWriter writer, - final DataSink dataSink) throws DbException { + public ListenableFuture startAggregatedSentLogDataStream( + final SubQueryId subqueryId, final TupleWriter writer, final DataSink dataSink) + throws DbException { Set actualWorkers = getWorkersForSubQuery(subqueryId); - final Schema schema = Schema.ofFields("fragmentId", Type.INT_TYPE, "numTuples", Type.LONG_TYPE, "minTime", - Type.LONG_TYPE, "maxTime", Type.LONG_TYPE); - - String sentQueryString = Joiner.on(' ').join( - "SELECT \"fragmentId\", sum(\"numTuples\") as \"numTuples\", min(\"nanoTime\") as \"minTime\", max(\"nanoTime\") as \"maxTime\" FROM", - MyriaConstants.SENT_PROFILING_RELATION.toString(getDBMS()), "WHERE \"queryId\" =", subqueryId.getQueryId(), - "AND \"subQueryId\" =", subqueryId.getSubqueryId(), "GROUP BY \"fragmentId\""); + final Schema schema = + Schema.ofFields( + "fragmentId", + Type.INT_TYPE, + "numTuples", + Type.LONG_TYPE, + "minTime", + Type.LONG_TYPE, + "maxTime", + Type.LONG_TYPE); + + String sentQueryString = + Joiner.on(' ') + .join( + "SELECT \"fragmentId\", sum(\"numTuples\") as \"numTuples\", min(\"nanoTime\") as \"minTime\", max(\"nanoTime\") as \"maxTime\" FROM", + MyriaConstants.SENT_PROFILING_RELATION.toString(getDBMS()), + "WHERE \"queryId\" =", + subqueryId.getQueryId(), + "AND \"subQueryId\" =", + subqueryId.getSubqueryId(), + "GROUP BY \"fragmentId\""); DbQueryScan scan = new DbQueryScan(sentQueryString, schema); final ExchangePairID operatorId = ExchangePairID.newID(); @@ -1492,27 +1710,38 @@ public ListenableFuture startAggregatedSentLogDataStream(final SubQueryId workerPlans.put(worker, workerPlan); } - final Consumer consumer = new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); - final SingleGroupByAggregate aggregate = new SingleGroupByAggregate(consumer, 0, - new SingleColumnAggregatorFactory(1, AggregationOp.SUM), - new SingleColumnAggregatorFactory(2, AggregationOp.MIN), - new SingleColumnAggregatorFactory(3, AggregationOp.MAX)); + final SingleGroupByAggregate aggregate = + new SingleGroupByAggregate( + consumer, + 0, + new SingleColumnAggregatorFactory(1, AggregationOp.SUM), + new SingleColumnAggregatorFactory(2, AggregationOp.MIN), + new SingleColumnAggregatorFactory(3, AggregationOp.MAX)); // rename columns ImmutableList.Builder renameExpressions = ImmutableList.builder(); renameExpressions.add(new Expression("fragmentId", new VariableExpression(0))); renameExpressions.add(new Expression("numTuples", new VariableExpression(1))); - renameExpressions - .add(new Expression("duration", new MinusExpression(new VariableExpression(3), new VariableExpression(2)))); + renameExpressions.add( + new Expression( + "duration", new MinusExpression(new VariableExpression(3), new VariableExpression(2)))); final Apply rename = new Apply(aggregate, renameExpressions.build()); TupleSink output = new TupleSink(rename, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); /* Submit the plan for the download. */ - String planString = Joiner.on("").join("download profiling aggregated sent data for (query=", - subqueryId.getQueryId(), ", subquery=", subqueryId.getSubqueryId(), ")"); + String planString = + Joiner.on("") + .join( + "download profiling aggregated sent data for (query=", + subqueryId.getQueryId(), + ", subquery=", + subqueryId.getSubqueryId(), + ")"); try { return queryManager.submitQuery(planString, planString, planString, masterPlan, workerPlans); } catch (CatalogException e) { @@ -1532,22 +1761,45 @@ public ListenableFuture startAggregatedSentLogDataStream(final SubQueryId * @return profiling logs for the query. * @throws DbException if there is an error when accessing profiling logs. */ - public QueryFuture startLogDataStream(final SubQueryId subqueryId, final long fragmentId, final long start, - final long end, final long minSpanLength, final boolean onlyRootOperator, final TupleWriter writer, - final DataSink dataSink) throws DbException { + public QueryFuture startLogDataStream( + final SubQueryId subqueryId, + final long fragmentId, + final long start, + final long end, + final long minSpanLength, + final boolean onlyRootOperator, + final TupleWriter writer, + final DataSink dataSink) + throws DbException { Preconditions.checkArgument(start < end, "range cannot be negative"); - final Schema schema = Schema.ofFields("opId", Type.INT_TYPE, "startTime", Type.LONG_TYPE, "endTime", Type.LONG_TYPE, - "numTuples", Type.LONG_TYPE); + final Schema schema = + Schema.ofFields( + "opId", + Type.INT_TYPE, + "startTime", + Type.LONG_TYPE, + "endTime", + Type.LONG_TYPE, + "numTuples", + Type.LONG_TYPE); Set actualWorkers = getWorkersForSubQuery(subqueryId); String opCondition = ""; if (onlyRootOperator) { - opCondition = Joiner.on(' ').join("AND \"opId\" = (SELECT \"opId\" FROM", - MyriaConstants.EVENT_PROFILING_RELATION.toString(getDBMS()), "WHERE \"fragmentId\" =", fragmentId, - " AND \"queryId\"=", subqueryId.getQueryId(), "AND \"subQueryId\" =", subqueryId.getSubqueryId(), - "ORDER BY \"startTime\" ASC LIMIT 1)"); + opCondition = + Joiner.on(' ') + .join( + "AND \"opId\" = (SELECT \"opId\" FROM", + MyriaConstants.EVENT_PROFILING_RELATION.toString(getDBMS()), + "WHERE \"fragmentId\" =", + fragmentId, + " AND \"queryId\"=", + subqueryId.getQueryId(), + "AND \"subQueryId\" =", + subqueryId.getSubqueryId(), + "ORDER BY \"startTime\" ASC LIMIT 1)"); } String spanCondition = ""; @@ -1555,11 +1807,24 @@ public QueryFuture startLogDataStream(final SubQueryId subqueryId, final long fr spanCondition = Joiner.on(' ').join("AND \"endTime\" - \"startTime\" >", minSpanLength); } - String queryString = Joiner.on(' ').join("SELECT \"opId\", \"startTime\", \"endTime\", \"numTuples\" FROM", - MyriaConstants.EVENT_PROFILING_RELATION.toString(getDBMS()), "WHERE \"fragmentId\" =", fragmentId, - "AND \"queryId\" =", subqueryId.getQueryId(), "AND \"subQueryId\" =", subqueryId.getSubqueryId(), - "AND \"endTime\" >", start, "AND \"startTime\" <", end, opCondition, spanCondition, - "ORDER BY \"startTime\" ASC"); + String queryString = + Joiner.on(' ') + .join( + "SELECT \"opId\", \"startTime\", \"endTime\", \"numTuples\" FROM", + MyriaConstants.EVENT_PROFILING_RELATION.toString(getDBMS()), + "WHERE \"fragmentId\" =", + fragmentId, + "AND \"queryId\" =", + subqueryId.getQueryId(), + "AND \"subQueryId\" =", + subqueryId.getSubqueryId(), + "AND \"endTime\" >", + start, + "AND \"startTime\" <", + end, + opCondition, + spanCondition, + "ORDER BY \"startTime\" ASC"); DbQueryScan scan = new DbQueryScan(queryString, schema); @@ -1576,7 +1841,8 @@ public QueryFuture startLogDataStream(final SubQueryId subqueryId, final long fr final ExchangePairID operatorId = ExchangePairID.newID(); - CollectProducer producer = new CollectProducer(addWorkerId, operatorId, MyriaConstants.MASTER_ID); + CollectProducer producer = + new CollectProducer(addWorkerId, operatorId, MyriaConstants.MASTER_ID); SubQueryPlan workerPlan = new SubQueryPlan(producer); Map workerPlans = new HashMap<>(actualWorkers.size()); @@ -1584,14 +1850,26 @@ public QueryFuture startLogDataStream(final SubQueryId subqueryId, final long fr workerPlans.put(worker, workerPlan); } - final Consumer consumer = new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); TupleSink output = new TupleSink(consumer, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); /* Submit the plan for the download. */ - String planString = Joiner.on("").join("download profiling data (query=", subqueryId.getQueryId(), ", subquery=", - subqueryId.getSubqueryId(), ", fragment=", fragmentId, ", range=[", Joiner.on(", ").join(start, end), "]", ")"); + String planString = + Joiner.on("") + .join( + "download profiling data (query=", + subqueryId.getQueryId(), + ", subquery=", + subqueryId.getSubqueryId(), + ", fragment=", + fragmentId, + ", range=[", + Joiner.on(", ").join(start, end), + "]", + ")"); try { return queryManager.submitQuery(planString, planString, planString, masterPlan, workerPlans); } catch (CatalogException e) { @@ -1614,14 +1892,22 @@ public QueryFuture startLogDataStream(final SubQueryId subqueryId, final long fr * @return profiling logs for the query. * @throws DbException if there is an error when accessing profiling logs. */ - public QueryFuture startHistogramDataStream(final SubQueryId subqueryId, final long fragmentId, final long start, - final long end, final long step, final boolean onlyRootOp, final TupleWriter writer, final DataSink dataSink) + public QueryFuture startHistogramDataStream( + final SubQueryId subqueryId, + final long fragmentId, + final long start, + final long end, + final long step, + final boolean onlyRootOp, + final TupleWriter writer, + final DataSink dataSink) throws DbException { Preconditions.checkArgument(start < end, "range cannot be negative"); Preconditions.checkArgument(step > 0, "step has to be greater than 0"); long bins = (end - start + 1) / step; - Preconditions.checkArgument(bins > 0 && bins <= MAX_BINS, "bins must be in the range [1, %s]", MAX_BINS); + Preconditions.checkArgument( + bins > 0 && bins <= MAX_BINS, "bins must be in the range [1, %s]", MAX_BINS); Set actualWorkers = getWorkersForSubQuery(subqueryId); @@ -1642,23 +1928,29 @@ public QueryFuture startHistogramDataStream(final SubQueryId subqueryId, final l String filterOpnameQueryString = ""; if (onlyRootOp) { sub = new StrSubstitutor(queryArgs); - filterOpnameQueryString = sub.replace( - "AND p.\"opId\"=(SELECT \"opId\" FROM ${PROF_TABLE} WHERE \"fragmentId\"=${FRAGMENT} AND \"queryId\"=${QUERY} AND \"subQueryId\"=${SUBQUERY} ORDER BY \"startTime\" ASC LIMIT 1)"); + filterOpnameQueryString = + sub.replace( + "AND p.\"opId\"=(SELECT \"opId\" FROM ${PROF_TABLE} WHERE \"fragmentId\"=${FRAGMENT} AND \"queryId\"=${QUERY} AND \"subQueryId\"=${SUBQUERY} ORDER BY \"startTime\" ASC LIMIT 1)"); } // Reinitialize the substitutor after including the opname filter. queryArgs.put("OPNAME_FILTER", filterOpnameQueryString); sub = new StrSubstitutor(queryArgs); - String histogramWorkerQueryString = sub.replace( - Joiner.on("\n").join("SELECT \"opId\", ${START}::bigint+${STEP}::bigint*s.bin as \"nanoTime\"", "FROM (", - "SELECT p.\"opId\", greatest((p.\"startTime\"-1-${START}::bigint)/${STEP}::bigint, -1) as \"startBin\", least((p.\"endTime\"+1-${START}::bigint)/${STEP}::bigint, ${BINS}) AS \"endBin\"", - "FROM ${PROF_TABLE} p", - "WHERE p.\"queryId\" = ${QUERY} and p.\"subQueryId\" = ${SUBQUERY} and p.\"fragmentId\" = ${FRAGMENT}", - "${OPNAME_FILTER}", - "AND greatest((p.\"startTime\"-${START}::bigint)/${STEP}::bigint, -1) < least((p.\"endTime\"-${START}::bigint)/${STEP}::bigint, ${BINS}) AND p.\"startTime\" < ${END}::bigint AND p.\"endTime\" >= ${START}::bigint", - ") times,", "generate_series(0, ${BINS}) AS s(bin)", - "WHERE s.bin > times.\"startBin\" and s.bin <= times.\"endBin\";")); + String histogramWorkerQueryString = + sub.replace( + Joiner.on("\n") + .join( + "SELECT \"opId\", ${START}::bigint+${STEP}::bigint*s.bin as \"nanoTime\"", + "FROM (", + "SELECT p.\"opId\", greatest((p.\"startTime\"-1-${START}::bigint)/${STEP}::bigint, -1) as \"startBin\", least((p.\"endTime\"+1-${START}::bigint)/${STEP}::bigint, ${BINS}) AS \"endBin\"", + "FROM ${PROF_TABLE} p", + "WHERE p.\"queryId\" = ${QUERY} and p.\"subQueryId\" = ${SUBQUERY} and p.\"fragmentId\" = ${FRAGMENT}", + "${OPNAME_FILTER}", + "AND greatest((p.\"startTime\"-${START}::bigint)/${STEP}::bigint, -1) < least((p.\"endTime\"-${START}::bigint)/${STEP}::bigint, ${BINS}) AND p.\"startTime\" < ${END}::bigint AND p.\"endTime\" >= ${START}::bigint", + ") times,", + "generate_series(0, ${BINS}) AS s(bin)", + "WHERE s.bin > times.\"startBin\" and s.bin <= times.\"endBin\";")); DbQueryScan scan = new DbQueryScan(histogramWorkerQueryString, schema); final ExchangePairID operatorId = ExchangePairID.newID(); @@ -1672,11 +1964,13 @@ public QueryFuture startHistogramDataStream(final SubQueryId subqueryId, final l } /* Aggregate histogram on master */ - final Consumer consumer = new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); // sum up the number of workers working - final MultiGroupByAggregate sumAggregate = new MultiGroupByAggregate(consumer, new int[] { 0, 1 }, - new SingleColumnAggregatorFactory(1, AggregationOp.COUNT)); + final MultiGroupByAggregate sumAggregate = + new MultiGroupByAggregate( + consumer, new int[] {0, 1}, new SingleColumnAggregatorFactory(1, AggregationOp.COUNT)); // rename columns ImmutableList.Builder renameExpressions = ImmutableList.builder(); renameExpressions.add(new Expression("opId", new VariableExpression(0))); @@ -1688,9 +1982,19 @@ public QueryFuture startHistogramDataStream(final SubQueryId subqueryId, final l final SubQueryPlan masterPlan = new SubQueryPlan(output); /* Submit the plan for the download. */ - String planString = Joiner.on("").join("download profiling histogram (query=", subqueryId.getQueryId(), - ", subquery=", subqueryId.getSubqueryId(), ", fragment=", fragmentId, ", range=[", - Joiner.on(", ").join(start, end, step), "]", ")"); + String planString = + Joiner.on("") + .join( + "download profiling histogram (query=", + subqueryId.getQueryId(), + ", subquery=", + subqueryId.getSubqueryId(), + ", fragment=", + fragmentId, + ", range=[", + Joiner.on(", ").join(start, end, step), + "]", + ")"); try { return queryManager.submitQuery(planString, planString, planString, masterPlan, workerPlans); } catch (CatalogException e) { @@ -1706,16 +2010,28 @@ public QueryFuture startHistogramDataStream(final SubQueryId subqueryId, final l * @return profiling logs for the query. * @throws DbException if there is an error when accessing profiling logs. */ - public QueryFuture startRangeDataStream(final SubQueryId subqueryId, final long fragmentId, final TupleWriter writer, - final DataSink dataSink) throws DbException { + public QueryFuture startRangeDataStream( + final SubQueryId subqueryId, + final long fragmentId, + final TupleWriter writer, + final DataSink dataSink) + throws DbException { final Schema schema = Schema.ofFields("startTime", Type.LONG_TYPE, "endTime", Type.LONG_TYPE); final RelationKey relationKey = MyriaConstants.EVENT_PROFILING_RELATION; Set actualWorkers = getWorkersForSubQuery(subqueryId); - String opnameQueryString = Joiner.on(' ').join("SELECT min(\"startTime\"), max(\"endTime\") FROM", - relationKey.toString(getDBMS()), "WHERE \"queryId\"=", subqueryId.getQueryId(), "AND \"subQueryId\"=", - subqueryId.getSubqueryId(), "AND \"fragmentId\"=", fragmentId); + String opnameQueryString = + Joiner.on(' ') + .join( + "SELECT min(\"startTime\"), max(\"endTime\") FROM", + relationKey.toString(getDBMS()), + "WHERE \"queryId\"=", + subqueryId.getQueryId(), + "AND \"subQueryId\"=", + subqueryId.getSubqueryId(), + "AND \"fragmentId\"=", + fragmentId); DbQueryScan scan = new DbQueryScan(opnameQueryString, schema); final ExchangePairID operatorId = ExchangePairID.newID(); @@ -1729,18 +2045,30 @@ public QueryFuture startRangeDataStream(final SubQueryId subqueryId, final long } /* Construct the master plan. */ - final Consumer consumer = new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); // Aggregate range on master - final Aggregate sumAggregate = new Aggregate(consumer, new SingleColumnAggregatorFactory(0, AggregationOp.MIN), - new SingleColumnAggregatorFactory(1, AggregationOp.MAX)); + final Aggregate sumAggregate = + new Aggregate( + consumer, + new SingleColumnAggregatorFactory(0, AggregationOp.MIN), + new SingleColumnAggregatorFactory(1, AggregationOp.MAX)); TupleSink output = new TupleSink(sumAggregate, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); /* Submit the plan for the download. */ - String planString = Joiner.on("").join("download time range (query=", subqueryId.getQueryId(), ", subquery=", - subqueryId.getSubqueryId(), ", fragment=", fragmentId, ")"); + String planString = + Joiner.on("") + .join( + "download time range (query=", + subqueryId.getQueryId(), + ", subquery=", + subqueryId.getSubqueryId(), + ", fragment=", + fragmentId, + ")"); try { return queryManager.submitQuery(planString, planString, planString, masterPlan, workerPlans); } catch (CatalogException e) { @@ -1756,8 +2084,12 @@ public QueryFuture startRangeDataStream(final SubQueryId subqueryId, final long * @return contributions for operator. * @throws DbException if there is an error when accessing profiling logs. */ - public QueryFuture startContributionsStream(final SubQueryId subqueryId, final long fragmentId, - final TupleWriter writer, final DataSink dataSink) throws DbException { + public QueryFuture startContributionsStream( + final SubQueryId subqueryId, + final long fragmentId, + final TupleWriter writer, + final DataSink dataSink) + throws DbException { final Schema schema = Schema.ofFields("opId", Type.INT_TYPE, "nanoTime", Type.LONG_TYPE); final RelationKey relationKey = MyriaConstants.EVENT_PROFILING_RELATION; @@ -1768,9 +2100,17 @@ public QueryFuture startContributionsStream(final SubQueryId subqueryId, final l fragIdCondition = "AND \"fragmentId\"=" + fragmentId; } - String opContributionsQueryString = Joiner.on(' ').join("SELECT \"opId\", sum(\"endTime\" - \"startTime\") FROM ", - relationKey.toString(getDBMS()), "WHERE \"queryId\"=", subqueryId.getQueryId(), "AND \"subQueryId\"=", - subqueryId.getSubqueryId(), fragIdCondition, "GROUP BY \"opId\""); + String opContributionsQueryString = + Joiner.on(' ') + .join( + "SELECT \"opId\", sum(\"endTime\" - \"startTime\") FROM ", + relationKey.toString(getDBMS()), + "WHERE \"queryId\"=", + subqueryId.getQueryId(), + "AND \"subQueryId\"=", + subqueryId.getSubqueryId(), + fragIdCondition, + "GROUP BY \"opId\""); DbQueryScan scan = new DbQueryScan(opContributionsQueryString, schema); final ExchangePairID operatorId = ExchangePairID.newID(); @@ -1784,11 +2124,13 @@ public QueryFuture startContributionsStream(final SubQueryId subqueryId, final l } /* Aggregate on master */ - final Consumer consumer = new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(scan.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); // sum up contributions - final SingleGroupByAggregate sumAggregate = new SingleGroupByAggregate(consumer, 0, - new SingleColumnAggregatorFactory(1, AggregationOp.AVG)); + final SingleGroupByAggregate sumAggregate = + new SingleGroupByAggregate( + consumer, 0, new SingleColumnAggregatorFactory(1, AggregationOp.AVG)); // rename columns ImmutableList.Builder renameExpressions = ImmutableList.builder(); @@ -1800,8 +2142,16 @@ public QueryFuture startContributionsStream(final SubQueryId subqueryId, final l final SubQueryPlan masterPlan = new SubQueryPlan(output); /* Submit the plan for the download. */ - String planString = Joiner.on("").join("download operator contributions (query=", subqueryId.getQueryId(), - ", subquery=", subqueryId.getSubqueryId(), ", fragment=", fragmentId, ")"); + String planString = + Joiner.on("") + .join( + "download operator contributions (query=", + subqueryId.getQueryId(), + ", subquery=", + subqueryId.getSubqueryId(), + ", fragment=", + fragmentId, + ")"); try { return queryManager.submitQuery(planString, planString, planString, masterPlan, workerPlans); } catch (CatalogException e) { @@ -1816,7 +2166,8 @@ public QueryFuture startContributionsStream(final SubQueryId subqueryId, final l * @param count the number of tuples in that relation * @throws DbException if there is an error in the catalog */ - public void updateRelationTupleCount(final RelationKey relation, final long count) throws DbException { + public void updateRelationTupleCount(final RelationKey relation, final long count) + throws DbException { try { catalog.updateRelationTupleCount(relation, count); } catch (CatalogException e) { @@ -1831,7 +2182,8 @@ public void updateRelationTupleCount(final RelationKey relation, final long coun * @param key the name of the variable * @param value the new value for the variable */ - public void setQueryGlobal(final long queryId, @Nonnull final String key, @Nonnull final Object value) { + public void setQueryGlobal( + final long queryId, @Nonnull final String key, @Nonnull final Object value) { Preconditions.checkNotNull(key, "key"); Preconditions.checkNotNull(value, "value"); queryManager.getQuery(queryId).setGlobal(key, value); @@ -1867,7 +2219,8 @@ public Schema getTempSchema(@Nonnull final Long queryId, @Nonnull final String n * @throws DbException if there is an error in the database. */ public void getResourceUsage(final long queryId, final DataSink dataSink) throws DbException { - Schema schema = Schema.appendColumn(MyriaConstants.RESOURCE_PROFILING_SCHEMA, Type.INT_TYPE, "workerId"); + Schema schema = + Schema.appendColumn(MyriaConstants.RESOURCE_PROFILING_SCHEMA, Type.INT_TYPE, "workerId"); try { TupleWriter writer = new CsvTupleWriter(); TupleBuffer tb = queryManager.getResourceUsage(queryId); @@ -1890,8 +2243,8 @@ public void getResourceUsage(final long queryId, final DataSink dataSink) throws * @return resource logs for the query. * @throws DbException if there is an error when accessing profiling logs. */ - public ListenableFuture getResourceLog(final long queryId, final TupleWriter writer, final DataSink dataSink) - throws DbException { + public ListenableFuture getResourceLog( + final long queryId, final TupleWriter writer, final DataSink dataSink) throws DbException { SubQueryId sqId = new SubQueryId(queryId, 0); String serializedPlan; try { @@ -1899,12 +2252,18 @@ public ListenableFuture getResourceLog(final long queryId, final TupleWri } catch (CatalogException e) { throw new DbException(e); } - Preconditions.checkArgument(serializedPlan != null, "No cached query plan for subquery %s", sqId); + Preconditions.checkArgument( + serializedPlan != null, "No cached query plan for subquery %s", sqId); Set actualWorkers = getWorkersFromSubqueryPlan(serializedPlan); final Schema schema = MyriaConstants.RESOURCE_PROFILING_SCHEMA; - String resourceQueryString = Joiner.on(' ').join("SELECT * from", - MyriaConstants.RESOURCE_PROFILING_RELATION.toString(getDBMS()), "WHERE \"queryId\" =", queryId); + String resourceQueryString = + Joiner.on(' ') + .join( + "SELECT * from", + MyriaConstants.RESOURCE_PROFILING_RELATION.toString(getDBMS()), + "WHERE \"queryId\" =", + queryId); DbQueryScan scan = new DbQueryScan(resourceQueryString, schema); ImmutableList.Builder emitExpressions = ImmutableList.builder(); @@ -1916,13 +2275,15 @@ public ListenableFuture getResourceLog(final long queryId, final TupleWri Apply addWorkerId = new Apply(scan, emitExpressions.build()); final ExchangePairID operatorId = ExchangePairID.newID(); - CollectProducer producer = new CollectProducer(addWorkerId, operatorId, MyriaConstants.MASTER_ID); + CollectProducer producer = + new CollectProducer(addWorkerId, operatorId, MyriaConstants.MASTER_ID); SubQueryPlan workerPlan = new SubQueryPlan(producer); Map workerPlans = new HashMap<>(actualWorkers.size()); for (Integer worker : actualWorkers) { workerPlans.put(worker, workerPlan); } - final Consumer consumer = new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); + final Consumer consumer = + new Consumer(addWorkerId.getSchema(), operatorId, ImmutableSet.copyOf(actualWorkers)); TupleSink output = new TupleSink(consumer, writer, dataSink); final SubQueryPlan masterPlan = new SubQueryPlan(output); @@ -1943,7 +2304,8 @@ public ListenableFuture getResourceLog(final long queryId, final TupleWri * @param encodedPlan the plan. * @throws DbException if there is an error in the catalog. */ - public void setQueryPlan(final SubQueryId subQueryId, @Nonnull final String encodedPlan) throws DbException { + public void setQueryPlan(final SubQueryId subQueryId, @Nonnull final String encodedPlan) + throws DbException { try { catalog.setQueryPlan(subQueryId, encodedPlan); } catch (CatalogException e) { diff --git a/src/edu/washington/escience/myria/util/HashUtils.java b/src/edu/washington/escience/myria/util/HashUtils.java index 615930755..0eb282993 100644 --- a/src/edu/washington/escience/myria/util/HashUtils.java +++ b/src/edu/washington/escience/myria/util/HashUtils.java @@ -103,7 +103,7 @@ public static int hashSubRow(final ReadableTable table, final int[] hashColumns, * @param table the table containing the values to be hashed * @param hashColumns the columns to be hashed. Order matters * @param row the row containing the values to be hashed - * @param seedIndex the index of the chosen hashcode + * @param seedIndex the index of the chosen hash function * @return the hash code of the specified columns in the specified row of the given table */ public static int hashSubRow( From b57a2bcf41a9f43510f7307ad719938f12c4fe0c Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Fri, 30 Dec 2016 10:24:46 -0800 Subject: [PATCH 07/15] remove partitionToDestination from constructor --- .../network/distribute/BroadcastDistributeFunction.java | 2 +- .../operator/network/distribute/DistributeFunction.java | 5 +---- .../operator/network/distribute/HashDistributeFunction.java | 2 +- .../network/distribute/HyperCubeDistributeFunction.java | 5 ++--- .../network/distribute/IdentityDistributeFunction.java | 2 +- .../network/distribute/RoundRobinDistributeFunction.java | 2 +- 6 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java index 700250119..7d5381e2b 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java @@ -16,7 +16,7 @@ public final class BroadcastDistributeFunction extends DistributeFunction { */ @JsonCreator public BroadcastDistributeFunction() { - super(new SinglePartitionFunction(), null); + super(new SinglePartitionFunction()); } /** diff --git a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java index 9a4a58a24..19f372772 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java @@ -42,12 +42,9 @@ public abstract class DistributeFunction implements Serializable { /** * @param partitionFunction partition function. - * @param partitionToDestination mapping from partitions to destinations */ - public DistributeFunction( - final PartitionFunction partitionFunction, final List> partitionToDestination) { + public DistributeFunction(final PartitionFunction partitionFunction) { this.partitionFunction = partitionFunction; - this.partitionToDestination = partitionToDestination; } /** diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java index ace1078db..577a3e369 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java @@ -20,7 +20,7 @@ public final class HashDistributeFunction extends DistributeFunction { */ @JsonCreator public HashDistributeFunction(@JsonProperty("indexes") final int[] indexes) { - super(new HashPartitionFunction(indexes), null); + super(new HashPartitionFunction(indexes)); this.indexes = indexes; } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java index ad3f11a1b..2ba31aa34 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java @@ -24,9 +24,8 @@ public HyperCubeDistributeFunction( @JsonProperty("hashedColumns") final int[] hashedColumns, @JsonProperty("mappedHCDimensions") final int[] mappedHCDimensions, @JsonProperty("cellPartition") final List> cellPartition) { - super( - new HyperCubePartitionFunction(hyperCubeDimensions, hashedColumns, mappedHCDimensions), - cellPartition); + super(new HyperCubePartitionFunction(hyperCubeDimensions, hashedColumns, mappedHCDimensions)); + partitionToDestination = cellPartition; } /** @return all destinations */ diff --git a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java index 930939d89..77187b36a 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java @@ -18,7 +18,7 @@ public final class IdentityDistributeFunction extends DistributeFunction { */ @JsonCreator public IdentityDistributeFunction(@JsonProperty("index") final int index) { - super(new IdentityPartitionFunction(index), null); + super(new IdentityPartitionFunction(index)); } @Override diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java index 7b54b3537..d20014d99 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java @@ -16,7 +16,7 @@ public final class RoundRobinDistributeFunction extends DistributeFunction { */ @JsonCreator public RoundRobinDistributeFunction() { - super(new RoundRobinPartitionFunction(), null); + super(new RoundRobinPartitionFunction()); } @Override From efb3a95eb403fb52a329c727f33a1ee1900d10ce Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Fri, 30 Dec 2016 10:28:35 -0800 Subject: [PATCH 08/15] fix names --- .../operator/network/RecoverProducer.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/src/edu/washington/escience/myria/operator/network/RecoverProducer.java b/src/edu/washington/escience/myria/operator/network/RecoverProducer.java index 9625e76d5..8b8edd82b 100644 --- a/src/edu/washington/escience/myria/operator/network/RecoverProducer.java +++ b/src/edu/washington/escience/myria/operator/network/RecoverProducer.java @@ -23,53 +23,53 @@ public final class RecoverProducer extends CollectProducer { org.slf4j.LoggerFactory.getLogger(RecoverProducer.class); /** the original producer. */ - private final Producer oriProducer; + private final Producer origProducer; /** the channel index that this operator is recovering for. */ - private final int channelIndx; + private final int channelIdx; /** * @param child the child who provides data for this producer to distribute. * @param operatorID destination operator the data goes * @param consumerWorkerID destination worker the data goes. - * @param oriProducer the original producer. - * @param channelInx the channel index that this operator is recovering for. * + * @param origProducer the original producer. + * @param channelIdx the channel index that this operator is recovering for. * */ public RecoverProducer( final Operator child, final ExchangePairID operatorID, final int consumerWorkerID, - final Producer oriProducer, - final int channelInx) { + final Producer origProducer, + final int channelIdx) { super(child, operatorID, consumerWorkerID); - this.oriProducer = oriProducer; - this.channelIndx = channelInx; + this.origProducer = origProducer; + this.channelIdx = channelIdx; } @Override protected void childEOS() throws DbException { writePartitionsIntoChannels(null); Preconditions.checkArgument(getChild() instanceof BatchTupleSource); - if (!oriProducer.eos()) { + if (!origProducer.eos()) { StreamOutputChannel tmp = getChannels()[0]; if (LOGGER.isTraceEnabled()) { LOGGER.trace( "recovery task {} detach & attach channel {} old channel {} new channel {}", getOpName(), tmp.getID(), - oriProducer.getChannels()[channelIndx], + origProducer.getChannels()[channelIdx], tmp); } - oriProducer.getChannels()[channelIndx] = tmp; + origProducer.getChannels()[channelIdx] = tmp; /* have to do this otherwise the channel will be released in resourceManager.cleanup() */ getTaskResourceManager().removeOutputChannel(tmp); /* have to do this otherwise the channel will be released in Producer.cleanup() */ getChannels()[0] = null; /* set the channel to be available again */ - oriProducer.getChannelsAvail()[channelIndx] = true; + origProducer.getChannelsAvail()[channelIdx] = true; /* if the channel was disabled before crash, need to give the task a chance to enable it. */ - oriProducer.getTaskResourceManager().getFragment().notifyOutputEnabled(tmp.getID()); + origProducer.getTaskResourceManager().getFragment().notifyOutputEnabled(tmp.getID()); /* if the task has no new input, but needs to produce potential EOSs & push TBs in its buffers out. */ - oriProducer.getTaskResourceManager().getFragment().notifyNewInput(); + origProducer.getTaskResourceManager().getFragment().notifyNewInput(); } else { channelEnds(0); } From 54486a278619053c883ecb235c19c8fdd15ecc00 Mon Sep 17 00:00:00 2001 From: jingjingwang Date: Wed, 4 Jan 2017 00:34:19 -0800 Subject: [PATCH 09/15] derive numDestinations from partitionToDestinations --- .../encoding/BroadcastProducerEncoding.java | 13 ++++++----- .../GenericShuffleProducerEncoding.java | 2 +- .../operator/network/CollectProducer.java | 3 ++- .../network/GenericShuffleProducer.java | 9 +++++++- .../network/LocalMultiwayProducer.java | 3 ++- .../BroadcastDistributeFunction.java | 12 +--------- .../distribute/DistributeFunction.java | 22 ++++++++++++++----- .../distribute/HashDistributeFunction.java | 5 ++--- .../distribute/HashPartitionFunction.java | 4 ---- .../HyperCubeDistributeFunction.java | 5 +++-- .../IdentityDistributeFunction.java | 5 ++--- .../RoundRobinDistributeFunction.java | 5 ++--- .../escience/myria/parallel/Server.java | 2 +- 13 files changed, 48 insertions(+), 42 deletions(-) diff --git a/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java index 2b19c614f..c73b46f14 100644 --- a/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/BroadcastProducerEncoding.java @@ -14,10 +14,13 @@ public class BroadcastProducerEncoding extends AbstractProducerEncoding workerIds = getRealWorkerIds(); List operatorIds = getRealOperatorIds(); - distributeFunction.setNumDestinations(workerIds.size(), operatorIds.size()); + distributeFunction.setDestinations(workerIds.size(), operatorIds.size()); GenericShuffleProducer producer = new GenericShuffleProducer( null, diff --git a/src/edu/washington/escience/myria/operator/network/CollectProducer.java b/src/edu/washington/escience/myria/operator/network/CollectProducer.java index 28e9d9d60..e57d50449 100644 --- a/src/edu/washington/escience/myria/operator/network/CollectProducer.java +++ b/src/edu/washington/escience/myria/operator/network/CollectProducer.java @@ -24,6 +24,7 @@ public CollectProducer( child, new ExchangePairID[] {operatorID}, new int[] {consumerWorkerID}, - new BroadcastDistributeFunction(1)); + new BroadcastDistributeFunction()); + this.distributeFunction.setDestinations(1, 1); } } diff --git a/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java b/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java index 61b3960ad..5b3ffbfe3 100644 --- a/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java +++ b/src/edu/washington/escience/myria/operator/network/GenericShuffleProducer.java @@ -19,7 +19,7 @@ public class GenericShuffleProducer extends Producer { private static final long serialVersionUID = 1L; /** the distribute function. */ - private final DistributeFunction distributeFunction; + protected final DistributeFunction distributeFunction; /** * Shuffle to multiple operator IDs on multiple workers. The most generic constructor. @@ -69,4 +69,11 @@ protected final void childEOI() throws DbException { writePartitionsIntoChannels( distributeFunction.distribute(TupleBatch.eoiTupleBatch(getSchema()))); } + + /** + * @return the distribute function + */ + public final DistributeFunction getDistributeFunction() { + return distributeFunction; + } } diff --git a/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java b/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java index f9bea8b05..c1745ce39 100644 --- a/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java +++ b/src/edu/washington/escience/myria/operator/network/LocalMultiwayProducer.java @@ -20,6 +20,7 @@ public LocalMultiwayProducer(final Operator child, final ExchangePairID[] operat child, operatorIDs, new int[] {IPCConnectionPool.SELF_IPC_ID}, - new BroadcastDistributeFunction(operatorIDs.length)); + new BroadcastDistributeFunction()); + this.distributeFunction.setDestinations(1, operatorIDs.length); } } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java index 7d5381e2b..a80759062 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/BroadcastDistributeFunction.java @@ -19,18 +19,8 @@ public BroadcastDistributeFunction() { super(new SinglePartitionFunction()); } - /** - * @param numDestinations number of destination - */ - public BroadcastDistributeFunction(final int numDestinations) { - this(); - partitionToDestination = MyriaArrayUtils.create2DHorizontalIndexList(numDestinations); - this.numDestinations = numDestinations; - } - @Override - public void setNumDestinations(final int numWorker, final int numOperatorId) { + public void setDestinations(final int numWorker, final int numOperatorId) { partitionToDestination = MyriaArrayUtils.create2DHorizontalIndexList(numWorker * numOperatorId); - numDestinations = numWorker * numOperatorId; } } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java index 19f372772..33d0d40ff 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/DistributeFunction.java @@ -2,7 +2,9 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import javax.annotation.Nonnull; @@ -37,9 +39,6 @@ public abstract class DistributeFunction implements Serializable { /** The mapping from partitions to destinations. */ protected List> partitionToDestination; - /** number of destinations. */ - protected int numDestinations; - /** * @param partitionFunction partition function. */ @@ -54,11 +53,11 @@ public DistributeFunction(final PartitionFunction partitionFunction) { public List> distribute(@Nonnull final TupleBatch data) { List> result = new ArrayList>(); if (data.isEOI()) { - for (int i = 0; i < numDestinations; ++i) { + for (int i = 0; i < getNumDestinations(); ++i) { result.add(Lists.newArrayList(data)); } } else { - for (int i = 0; i < numDestinations; ++i) { + for (int i = 0; i < getNumDestinations(); ++i) { result.add(new ArrayList()); } TupleBatch[] tbs = partitionFunction.partition(data); @@ -71,9 +70,20 @@ public List> distribute(@Nonnull final TupleBatch data) { return result; } + /** + * @return number of destinations + */ + public int getNumDestinations() { + Set d = new HashSet(); + for (List t : partitionToDestination) { + d.addAll(t); + } + return d.size(); + } + /** * @param numWorker the number of workers to distribute on * @param numOperatorId the number of involved operator IDs */ - public abstract void setNumDestinations(final int numWorker, final int numOperatorId); + public abstract void setDestinations(final int numWorker, final int numOperatorId); } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java index 577a3e369..e51a67d92 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HashDistributeFunction.java @@ -25,10 +25,9 @@ public HashDistributeFunction(@JsonProperty("indexes") final int[] indexes) { } @Override - public void setNumDestinations(final int numWorker, final int numOperatorId) { - partitionFunction.setNumPartitions(numWorker); + public void setDestinations(final int numWorker, final int numOperatorId) { partitionToDestination = MyriaArrayUtils.create2DVerticalIndexList(numWorker); - numDestinations = numWorker; + partitionFunction.setNumPartitions(numWorker); } /** diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java index 22bdf7760..d5c1f6e07 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HashPartitionFunction.java @@ -19,18 +19,14 @@ public final class HashPartitionFunction extends PartitionFunction { private static final long serialVersionUID = 1L; /** The indices used for partitioning. */ - // @JsonProperty private final int[] indexes; /** The index of the chosen hashcode in HashUtils. */ - // @JsonProperty private final int seedIndex; /** * @param indexes the indices used for partitioning. */ - // @JsonCreator - // public HashPartitionFunction(@JsonProperty("indexes") final int[] indexes) { public HashPartitionFunction(final int[] indexes) { this(indexes, 0); } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java index 2ba31aa34..505180f08 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/HyperCubeDistributeFunction.java @@ -38,7 +38,8 @@ public List getAllDestinations() { } @Override - public void setNumDestinations(final int numWorker, final int numOperatorId) { - numDestinations = getAllDestinations().size(); + public void setDestinations(int numWorker, int numOperatorId) { + partitionFunction.setNumPartitions(partitionToDestination.size()); + /* TODO: should this be the same as the product of the sizes of all dimensions? */ } } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java index 77187b36a..9a1e69ad6 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/IdentityDistributeFunction.java @@ -22,9 +22,8 @@ public IdentityDistributeFunction(@JsonProperty("index") final int index) { } @Override - public void setNumDestinations(final int numWorker, final int numOperatorId) { - partitionFunction.setNumPartitions(numWorker); + public void setDestinations(int numWorker, int numOperatorId) { partitionToDestination = MyriaArrayUtils.create2DVerticalIndexList(numWorker); - numDestinations = numWorker; + partitionFunction.setNumPartitions(numWorker); } } diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java index d20014d99..7673ebc6d 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinDistributeFunction.java @@ -20,9 +20,8 @@ public RoundRobinDistributeFunction() { } @Override - public void setNumDestinations(final int numWorker, final int numOperatorId) { - partitionFunction.setNumPartitions(numWorker); + public void setDestinations(final int numWorker, final int numOperatorId) { partitionToDestination = MyriaArrayUtils.create2DVerticalIndexList(numWorker); - numDestinations = numWorker; + partitionFunction.setNumPartitions(numWorker); } } diff --git a/src/edu/washington/escience/myria/parallel/Server.java b/src/edu/washington/escience/myria/parallel/Server.java index a0f5f4a29..7e8fc31f8 100644 --- a/src/edu/washington/escience/myria/parallel/Server.java +++ b/src/edu/washington/escience/myria/parallel/Server.java @@ -818,7 +818,7 @@ public DatasetStatus ingestDataset( /* The master plan: send the tuples out. */ ExchangePairID scatterId = ExchangePairID.newID(); - df.setNumDestinations(workersArray.length, 1); + df.setDestinations(workersArray.length, 1); GenericShuffleProducer scatter = new GenericShuffleProducer(source, new ExchangePairID[] {scatterId}, workersArray, df); From ed8f118c6c15b80fc7ff5a9a65b348da1ccdc4e2 Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Wed, 4 Jan 2017 12:11:42 -0800 Subject: [PATCH 10/15] track source relations in QueryScanEncoding and assign workers accordingly --- .../myria/api/encoding/QueryConstruct.java | 16 ++++++++++++++++ .../myria/api/encoding/QueryScanEncoding.java | 4 ++++ 2 files changed, 20 insertions(+) diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index 75530cf62..9317e39c1 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -8,6 +8,7 @@ import javax.annotation.Nonnull; import javax.ws.rs.core.Response.Status; +import com.google.common.base.Joiner; import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.collect.ArrayListMultimap; @@ -198,6 +199,21 @@ private static void setAndVerifyScans( .getQueryManager() .getWorkersForTempRelation( args.getQueryId(), RelationKey.ofTemp(args.getQueryId(), scan.table)); + } else if (operator instanceof QueryScanEncoding) { + QueryScanEncoding scan = ((QueryScanEncoding) operator); + scanWorkers = server.getAliveWorkers(); + scanRelation = "(source relations for query scan):"; + for (RelationKey relationKey : scan.sourceRelationKeys) { + scanRelation += " " + relationKey.toString(); + Set workersForRelation = server.getWorkersForRelation(relationKey, null); + // REVIEW: This logic will work for broadcast relations stored on + // distinct but overlapping sets of workers, but where will it break? + scanWorkers = Sets.intersection(workersForRelation, scanWorkers); + } + LOGGER.info( + "DbQueryScan operator for relations {} assigned to workers {}", + scanRelation, + Joiner.on(',').join(scanWorkers)); } else { continue; } diff --git a/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java b/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java index 945604f7b..21b9d923f 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java @@ -1,5 +1,8 @@ package edu.washington.escience.myria.api.encoding; +import java.util.List; + +import edu.washington.escience.myria.RelationKey; import edu.washington.escience.myria.Schema; import edu.washington.escience.myria.api.encoding.QueryConstruct.ConstructArgs; import edu.washington.escience.myria.operator.DbQueryScan; @@ -7,6 +10,7 @@ public class QueryScanEncoding extends LeafOperatorEncoding { @Required public Schema schema; @Required public String sql; + @Required public List sourceRelationKeys; @Override public DbQueryScan construct(ConstructArgs args) { From 33540c9f45d910564067807ee73c96cbafba53dd Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Wed, 4 Jan 2017 16:48:57 -0800 Subject: [PATCH 11/15] use strict intersection of scan workers --- .../myria/api/encoding/QueryConstruct.java | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index 9317e39c1..04632806b 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -184,7 +184,7 @@ private static void setAndVerifyScans( for (PlanFragmentEncoding fragment : fragments) { for (OperatorEncoding operator : fragment.operators) { - Set scanWorkers; + Set scanWorkers = ImmutableSet.of(); String scanRelation; if (operator instanceof TableScanEncoding) { @@ -201,14 +201,23 @@ private static void setAndVerifyScans( args.getQueryId(), RelationKey.ofTemp(args.getQueryId(), scan.table)); } else if (operator instanceof QueryScanEncoding) { QueryScanEncoding scan = ((QueryScanEncoding) operator); - scanWorkers = server.getAliveWorkers(); scanRelation = "(source relations for query scan):"; + int relationIdx = 0; for (RelationKey relationKey : scan.sourceRelationKeys) { scanRelation += " " + relationKey.toString(); Set workersForRelation = server.getWorkersForRelation(relationKey, null); + // Guava's set operations don't accept null + if (workersForRelation == null) { + workersForRelation = ImmutableSet.of(); + } // REVIEW: This logic will work for broadcast relations stored on // distinct but overlapping sets of workers, but where will it break? - scanWorkers = Sets.intersection(workersForRelation, scanWorkers); + if (relationIdx == 0) { + scanWorkers = workersForRelation; + } else { + scanWorkers = Sets.intersection(workersForRelation, scanWorkers); + } + ++relationIdx; } LOGGER.info( "DbQueryScan operator for relations {} assigned to workers {}", @@ -218,7 +227,9 @@ private static void setAndVerifyScans( continue; } Preconditions.checkArgument( - scanWorkers != null, "Unable to find workers that store %s", scanRelation); + scanWorkers != null && !scanWorkers.isEmpty(), + "Unable to find workers that store %s", + scanRelation); /* * Note: the current assumption is that all the partitions need to be scanned. This will not be true if we have * data replication, or allow to scan only a subset of the partitions. Revise if needed. From 4083c95c07b33d91537cfbdd7471dd890095ce2b Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Wed, 4 Jan 2017 22:57:58 -0800 Subject: [PATCH 12/15] set singleton worker on debroadcast scans --- .../escience/myria/api/encoding/QueryConstruct.java | 9 +++++++++ .../escience/myria/api/encoding/QueryScanEncoding.java | 1 + .../escience/myria/api/encoding/TableScanEncoding.java | 1 + .../myria/api/encoding/TempTableScanEncoding.java | 1 + 4 files changed, 12 insertions(+) diff --git a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java index f215b486a..4838fd596 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryConstruct.java @@ -186,19 +186,23 @@ private static void setAndVerifyScans( for (OperatorEncoding operator : fragment.operators) { Set scanWorkers = ImmutableSet.of(); String scanRelation; + boolean debroadcast = false; if (operator instanceof TableScanEncoding) { TableScanEncoding scan = ((TableScanEncoding) operator); + debroadcast = scan.debroadcast; scanRelation = scan.relationKey.toString(); scanWorkers = server.getWorkersForRelation(scan.relationKey, scan.storedRelationId); } else if (operator instanceof TempTableScanEncoding) { TempTableScanEncoding scan = ((TempTableScanEncoding) operator); + debroadcast = scan.debroadcast; RelationKey relationKey = RelationKey.ofTemp(args.getQueryId(), scan.table); scanRelation = "temporary relation " + scan.table; scanWorkers = server.getQueryManager().getWorkersForTempRelation(args.getQueryId(), relationKey); } else if (operator instanceof QueryScanEncoding) { QueryScanEncoding scan = ((QueryScanEncoding) operator); + debroadcast = scan.debroadcast; scanRelation = "(source relations for query scan):"; int relationIdx = 0; for (RelationKey relationKey : scan.sourceRelationKeys) { @@ -228,6 +232,11 @@ private static void setAndVerifyScans( scanWorkers != null && !scanWorkers.isEmpty(), "Unable to find workers that store %s", scanRelation); + if (debroadcast) { + // we need to pick a single worker that is alive + Set aliveScanWorkers = Sets.intersection(scanWorkers, server.getAliveWorkers()); + scanWorkers = ImmutableSet.of(aliveScanWorkers.iterator().next()); + } /* * Note: the current assumption is that all the partitions need to be scanned. This will not be true if we have * data replication, or allow to scan only a subset of the partitions. Revise if needed. diff --git a/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java b/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java index 21b9d923f..ed3ebf8ec 100644 --- a/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/QueryScanEncoding.java @@ -11,6 +11,7 @@ public class QueryScanEncoding extends LeafOperatorEncoding { @Required public Schema schema; @Required public String sql; @Required public List sourceRelationKeys; + public boolean debroadcast; @Override public DbQueryScan construct(ConstructArgs args) { diff --git a/src/edu/washington/escience/myria/api/encoding/TableScanEncoding.java b/src/edu/washington/escience/myria/api/encoding/TableScanEncoding.java index d1ce6935a..31c286af2 100644 --- a/src/edu/washington/escience/myria/api/encoding/TableScanEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/TableScanEncoding.java @@ -16,6 +16,7 @@ public class TableScanEncoding extends LeafOperatorEncoding { /** The name of the relation to be scanned. */ @Required public RelationKey relationKey; public Integer storedRelationId; + public boolean debroadcast; @Override public DbQueryScan construct(ConstructArgs args) { diff --git a/src/edu/washington/escience/myria/api/encoding/TempTableScanEncoding.java b/src/edu/washington/escience/myria/api/encoding/TempTableScanEncoding.java index f454a1236..0c17aab81 100644 --- a/src/edu/washington/escience/myria/api/encoding/TempTableScanEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/TempTableScanEncoding.java @@ -11,6 +11,7 @@ public class TempTableScanEncoding extends LeafOperatorEncoding { /** The name of the relation to be scanned. */ @Required public String table; + public boolean debroadcast; @Override public DbQueryScan construct(ConstructArgs args) { From e33ccac00156f863704169e5dc6152f88d1762be Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Thu, 5 Jan 2017 15:30:11 -0800 Subject: [PATCH 13/15] logging --- .../network/distribute/RoundRobinPartitionFunction.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java index 1fd805f91..6205825f9 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java @@ -6,11 +6,15 @@ import edu.washington.escience.myria.storage.TupleBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * A partition function that simply sends one tuple to each output in turn. */ public final class RoundRobinPartitionFunction extends PartitionFunction { + private static final Logger LOGGER = LoggerFactory.getLogger(RoundRobinPartitionFunction.class); /** Required for Java serialization. */ private static final long serialVersionUID = 1L; /** The current partition to use. */ @@ -20,6 +24,7 @@ public final class RoundRobinPartitionFunction extends PartitionFunction { public TupleBatch[] partition(@Nonnull final TupleBatch tb) { BitSet[] partitions = new BitSet[numPartitions()]; for (int i = 0; i < tb.numTuples(); i++) { + LOGGER.info("Current partition: " + curPartition); partitions[curPartition].set(i); curPartition = (curPartition + 1) % numPartitions(); } From a4ee4074ceb90ce97b90918dfce2acd0657c20ef Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Thu, 5 Jan 2017 17:29:49 -0800 Subject: [PATCH 14/15] logging --- .../myria/api/encoding/GenericShuffleProducerEncoding.java | 7 +++++++ .../network/distribute/RoundRobinPartitionFunction.java | 1 + 2 files changed, 8 insertions(+) diff --git a/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java index 8d1a48493..487a937e7 100644 --- a/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java @@ -10,10 +10,15 @@ import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** JSON wrapper for GenericShuffleProducer encoding. */ public class GenericShuffleProducerEncoding extends AbstractProducerEncoding { + private static final Logger LOGGER = LoggerFactory.getLogger(GenericShuffleProducerEncoding.class); + /** The distribute function. */ @Required public DistributeFunction distributeFunction; @@ -23,7 +28,9 @@ public class GenericShuffleProducerEncoding @Override public GenericShuffleProducer construct(final ConstructArgs args) { Set workerIds = getRealWorkerIds(); + LOGGER.info("worker IDs: " + workerIds.toString()); List operatorIds = getRealOperatorIds(); + LOGGER.info("operator IDs: " + operatorIds.toString()); distributeFunction.setDestinations(workerIds.size(), operatorIds.size()); GenericShuffleProducer producer = new GenericShuffleProducer( diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java index 6205825f9..9ff245140 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java @@ -22,6 +22,7 @@ public final class RoundRobinPartitionFunction extends PartitionFunction { @Override public TupleBatch[] partition(@Nonnull final TupleBatch tb) { + LOGGER.info("Number of partitions: " + numPartitions()); BitSet[] partitions = new BitSet[numPartitions()]; for (int i = 0; i < tb.numTuples(); i++) { LOGGER.info("Current partition: " + curPartition); From 654be579f7bc75983ff53a0b656b4d6442a4b1e6 Mon Sep 17 00:00:00 2001 From: Tobin Baker Date: Thu, 5 Jan 2017 17:58:25 -0800 Subject: [PATCH 15/15] fix NPE --- .../api/encoding/GenericShuffleProducerEncoding.java | 7 ------- .../network/distribute/RoundRobinPartitionFunction.java | 9 +++------ 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java index 487a937e7..8d1a48493 100644 --- a/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java +++ b/src/edu/washington/escience/myria/api/encoding/GenericShuffleProducerEncoding.java @@ -10,15 +10,10 @@ import edu.washington.escience.myria.operator.network.distribute.DistributeFunction; import edu.washington.escience.myria.parallel.ExchangePairID; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** JSON wrapper for GenericShuffleProducer encoding. */ public class GenericShuffleProducerEncoding extends AbstractProducerEncoding { - private static final Logger LOGGER = LoggerFactory.getLogger(GenericShuffleProducerEncoding.class); - /** The distribute function. */ @Required public DistributeFunction distributeFunction; @@ -28,9 +23,7 @@ public class GenericShuffleProducerEncoding @Override public GenericShuffleProducer construct(final ConstructArgs args) { Set workerIds = getRealWorkerIds(); - LOGGER.info("worker IDs: " + workerIds.toString()); List operatorIds = getRealOperatorIds(); - LOGGER.info("operator IDs: " + operatorIds.toString()); distributeFunction.setDestinations(workerIds.size(), operatorIds.size()); GenericShuffleProducer producer = new GenericShuffleProducer( diff --git a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java index 9ff245140..ce7110ead 100644 --- a/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java +++ b/src/edu/washington/escience/myria/operator/network/distribute/RoundRobinPartitionFunction.java @@ -6,15 +6,11 @@ import edu.washington.escience.myria.storage.TupleBatch; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * A partition function that simply sends one tuple to each output in turn. */ public final class RoundRobinPartitionFunction extends PartitionFunction { - private static final Logger LOGGER = LoggerFactory.getLogger(RoundRobinPartitionFunction.class); /** Required for Java serialization. */ private static final long serialVersionUID = 1L; /** The current partition to use. */ @@ -22,10 +18,11 @@ public final class RoundRobinPartitionFunction extends PartitionFunction { @Override public TupleBatch[] partition(@Nonnull final TupleBatch tb) { - LOGGER.info("Number of partitions: " + numPartitions()); BitSet[] partitions = new BitSet[numPartitions()]; + for (int i = 0; i < partitions.length; ++i) { + partitions[i] = new BitSet(); + } for (int i = 0; i < tb.numTuples(); i++) { - LOGGER.info("Current partition: " + curPartition); partitions[curPartition].set(i); curPartition = (curPartition + 1) % numPartitions(); }