You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@apex.apache.org by ilganeli <gi...@git.apache.org> on 2016/06/08 19:10:16 UTC

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102][WIP] Support clone partitio...

GitHub user ilganeli opened a pull request:

    https://github.com/apache/apex-malhar/pull/315

    [APEXMALHAR-2102][WIP] Support clone partitioner

    Created a Clone partitioner similar to the StatelessPartitioner that assigns all data to all partitions
    Added a simple unit test suite to test scale up and scale down
    Added a unit test that adds the operator to a DAG and demonstrates that all parittions get the same data
    
    This references: https://issues.apache.org/jira/browse/APEXMALHAR-2102

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ilganeli/incubator-apex-malhar APEXCORE-146B

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/apex-malhar/pull/315.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #315
    
----
commit 74f6fd5113a3d488015a9817790018dd63b5e8cb
Author: Ilya Ganelin <il...@capitalone.com>
Date:   2016-05-26T19:39:24Z

    Added a new partitioner that replicates data across all partitions by default.

commit aef60461c8d43c8c7cbfa997a5c9cbe185301165
Author: Ilya Ganelin <il...@capitalone.com>
Date:   2016-05-26T19:39:24Z

    Added a new partitioner that replicates data across all partitions by default.
    
    Fixed package settings. Deleted unused imports.

commit 580f4af7491842bdf13ca7375dc12c3252f578c8
Author: Ilya Ganelin <il...@capitalone.com>
Date:   2016-05-26T20:37:28Z

    Merge conflicts resolved.

commit 3d127607c0d0757b68b16b275d64f4c8d1cb73d7
Author: Ilya Ganelin <il...@capitalone.com>
Date:   2016-05-26T19:39:24Z

    Added a new partitioner that replicates data across all partitions by default.
    
    Fixed package settings. Deleted unused imports.
    
    Added a test that demonstrates all partitions seeing the same data.
    
    Fixed test failure.
    
    Updated unit test to actually create an application using the ClonePartitioner and demonstrate that all partitions of an operator see the same data.
    
    Cleande up tests.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66347754
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,143 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  private static final long serialVersionUID = 201462371710L;
    +  /**
    +   * The number of partitions for the default partitioner to create.
    +   */
    +  @Min(1)
    +  private int partitionCount = 1;
    +
    +  /**
    +   * This creates a partitioner which creates only one partition.
    +   */
    +  public ClonePartitioner()
    +  {
    +  }
    +
    +  /**
    +   * This constructor is used to create the partitioner from a property.
    +   * @param value A string which is an integer of the number of partitions to create
    +   */
    +  public ClonePartitioner(String value)
    +  {
    +    this(Integer.parseInt(value));
    +  }
    +
    +  /**
    +   * This creates a partitioner which creates partitonCount partitions.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public ClonePartitioner(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method sets the number of partitions for the ClonePartitioner to create.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public void setPartitionCount(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method gets the number of partitions for the ClonePartitioner to create.
    +   * @return The number of partitions to create.
    +   */
    +  public int getPartitionCount()
    +  {
    +    return partitionCount;
    +  }
    +
    +  @Override
    +  public Collection<Partition<T>> definePartitions(Collection<Partition<T>> partitions, PartitioningContext context)
    +  {
    +    final int newPartitionCount = DefaultPartition.getRequiredPartitionCount(context, this.partitionCount);
    +    logger.debug("define partitions, partitionCount current {} requested {}", partitions.size(), newPartitionCount);
    +
    +    //Get a partition
    +    DefaultPartition<T> partition = (DefaultPartition<T>)partitions.iterator().next();
    +    ArrayList<Partition<T>> newPartitions;
    --- End diff --
    
    why not `ArrayList<Partition<T>> newPartitions = Lists.newArrayList();` and remove line 112?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66321909
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +  private static Integer totalReceived = 0;
    --- End diff --
    
    Is this variable required?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348833
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +
    +  private static boolean matchFailed = false;
    +
    +  public static class FixedEmitter extends BaseOperator implements InputOperator
    +  {
    +    public final DefaultOutputPort<Integer> output = new DefaultOutputPort<>();
    +
    +    private Integer emitted;
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      emitted = 0;
    +    }
    +
    +    @Override
    +    public void emitTuples()
    +    {
    +      while (emitted < countPerWindow) {
    +        output.emit(emitted++);
    +      }
    +    }
    +  }
    +
    +  public static class DummyOperator extends BaseOperator
    +  {
    +    private Integer value;
    +
    +    private transient ArrayList<Integer> valuesReceived = new ArrayList<>();
    +
    +    public DummyOperator()
    +    {
    +    }
    +
    +    public DummyOperator(Integer value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public final transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
    +    {
    +      @Override
    +      public void process(Integer tuple)
    +      {
    +        valuesReceived.add(tuple);
    +      }
    +    };
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      valuesReceived.clear();
    +    }
    +
    +    @Override
    +    public void endWindow()
    +    {
    +      if (valuesReceived.size() < countPerWindow) {
    +        matchFailed = true;
    +      } else {
    +        for (int i = 0; i < countPerWindow; i++) {
    +          if (valuesReceived.get(i) != i) {
    +            matchFailed = true;
    +          }
    +        }
    +      }
    +    }
    +
    +    public void setValue(int value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public int getValue()
    +    {
    +      return value;
    +    }
    +  }
    +
    +  @Test
    +  public void partition1Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 1, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void partition5Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>(5);
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 5, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void objectPropertyTest()
    +  {
    +    Object2String<ClonePartitioner<DummyOperator>> propertyReader = new Object2String<>();
    +    ClonePartitioner<DummyOperator> partitioner =
    +        propertyReader.fromString("com.datatorrent.lib.partitioner.ClonePartitioner:3");
    +    assertEquals(3, partitioner.getPartitionCount());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleUP()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    partitions.add(new DefaultPartition<>(dummyOperator));
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 5));
    +    assertEquals("after partition", 5, newPartitions.size());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleDown()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +
    +    for (int i = 5; i-- > 0; ) {
    +      partitions.add(new DefaultPartition<>(dummyOperator));
    +    }
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 1));
    +    assertEquals("after partition", 1, newPartitions.size());
    +  }
    +
    +  static class Application implements StreamingApplication
    +  {
    +    @Override
    +    public void populateDAG(DAG dag, Configuration conf)
    +    {
    +      LOG.debug("Application - PopulateDAG");
    +      FixedEmitter emitter = new FixedEmitter();
    +      DummyOperator dummy = new DummyOperator(5);
    +
    +      dag.addOperator("Emitter", emitter);
    +      dag.addOperator("Dummy", dummy);
    +
    +      ClonePartitioner<DummyOperator> partitioner = new ClonePartitioner<>();
    +      partitioner.setPartitionCount(numPartitions);
    +
    +      dag.setAttribute(dummy, Context.OperatorContext.PARTITIONER, partitioner);
    +      dag.addStream("Emitter to dummy", emitter.output, dummy.input);
    +    }
    +  }
    +
    +  @Test
    +  public void testCloningAcrossPartitions() throws IOException, Exception
    +  {
    +    try {
    +      LocalMode lma = LocalMode.newInstance();
    +      Configuration conf = new Configuration(false);
    +      lma.prepareDAG(new Application(), conf);
    +      LocalMode.Controller lc = lma.getController();
    +
    +      lc.run(20000); // runs for 10 seconds and quits
    +      assertFalse("Failed to match all values in all partitions.", matchFailed);
    +    } catch (ConstraintViolationException e) {
    +      Assert.fail("constraint violations: " + e.getConstraintViolations());
    +    }
    +  }
    +
    +
    +  public static class PartitioningContextImpl implements Partitioner.PartitioningContext
    +  {
    +    final int parallelPartitionCount;
    +    final List<InputPort<?>> ports;
    +
    +    public PartitioningContextImpl(List<InputPort<?>> ports, int parallelPartitionCount)
    +    {
    +      this.ports = ports;
    +      this.parallelPartitionCount = parallelPartitionCount;
    +    }
    +
    +    @Override
    +    public int getParallelPartitionCount()
    +    {
    +      return parallelPartitionCount;
    +    }
    +
    +    @Override
    +    public List<InputPort<?>> getInputPorts()
    +    {
    +      return ports;
    +    }
    +  }
    +
    --- End diff --
    
    nit: remove newline


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348710
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +
    +  private static boolean matchFailed = false;
    +
    +  public static class FixedEmitter extends BaseOperator implements InputOperator
    +  {
    +    public final DefaultOutputPort<Integer> output = new DefaultOutputPort<>();
    +
    +    private Integer emitted;
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      emitted = 0;
    +    }
    +
    +    @Override
    +    public void emitTuples()
    +    {
    +      while (emitted < countPerWindow) {
    +        output.emit(emitted++);
    +      }
    +    }
    +  }
    +
    +  public static class DummyOperator extends BaseOperator
    +  {
    +    private Integer value;
    +
    +    private transient ArrayList<Integer> valuesReceived = new ArrayList<>();
    +
    +    public DummyOperator()
    +    {
    +    }
    +
    +    public DummyOperator(Integer value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public final transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
    +    {
    +      @Override
    +      public void process(Integer tuple)
    +      {
    +        valuesReceived.add(tuple);
    +      }
    +    };
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      valuesReceived.clear();
    +    }
    +
    +    @Override
    +    public void endWindow()
    +    {
    +      if (valuesReceived.size() < countPerWindow) {
    +        matchFailed = true;
    +      } else {
    +        for (int i = 0; i < countPerWindow; i++) {
    +          if (valuesReceived.get(i) != i) {
    +            matchFailed = true;
    +          }
    +        }
    +      }
    +    }
    +
    +    public void setValue(int value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public int getValue()
    +    {
    +      return value;
    +    }
    +  }
    +
    +  @Test
    +  public void partition1Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 1, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void partition5Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>(5);
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 5, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void objectPropertyTest()
    +  {
    +    Object2String<ClonePartitioner<DummyOperator>> propertyReader = new Object2String<>();
    +    ClonePartitioner<DummyOperator> partitioner =
    +        propertyReader.fromString("com.datatorrent.lib.partitioner.ClonePartitioner:3");
    +    assertEquals(3, partitioner.getPartitionCount());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleUP()
    --- End diff --
    
    rename to `parallelPartitionScaleUpTest`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348114
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,143 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  private static final long serialVersionUID = 201462371710L;
    +  /**
    +   * The number of partitions for the default partitioner to create.
    +   */
    +  @Min(1)
    +  private int partitionCount = 1;
    +
    +  /**
    +   * This creates a partitioner which creates only one partition.
    +   */
    +  public ClonePartitioner()
    +  {
    +  }
    +
    +  /**
    +   * This constructor is used to create the partitioner from a property.
    +   * @param value A string which is an integer of the number of partitions to create
    +   */
    +  public ClonePartitioner(String value)
    +  {
    +    this(Integer.parseInt(value));
    +  }
    +
    +  /**
    +   * This creates a partitioner which creates partitonCount partitions.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public ClonePartitioner(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method sets the number of partitions for the ClonePartitioner to create.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public void setPartitionCount(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method gets the number of partitions for the ClonePartitioner to create.
    +   * @return The number of partitions to create.
    +   */
    +  public int getPartitionCount()
    +  {
    +    return partitionCount;
    +  }
    +
    +  @Override
    +  public Collection<Partition<T>> definePartitions(Collection<Partition<T>> partitions, PartitioningContext context)
    +  {
    +    final int newPartitionCount = DefaultPartition.getRequiredPartitionCount(context, this.partitionCount);
    +    logger.debug("define partitions, partitionCount current {} requested {}", partitions.size(), newPartitionCount);
    +
    +    //Get a partition
    +    DefaultPartition<T> partition = (DefaultPartition<T>)partitions.iterator().next();
    +    ArrayList<Partition<T>> newPartitions;
    +
    +    // first call to define partitions
    +    newPartitions = Lists.newArrayList();
    +
    +    Set partSet = Sets.newHashSet();
    +
    +    // Add as many partitions as we wish to have
    +    for (int partitionCounter = 0; partitionCounter < newPartitionCount; partitionCounter++) {
    +      DefaultPartition<T> partitionToAdd = new DefaultPartition<T>(partition.getPartitionedInstance());
    +      newPartitions.add(partitionToAdd);
    --- End diff --
    
    why not `newPartitions.add(new DefaultPartition<T>(partition.getPartitionedInstance()))` and collapse lines 118 / 119?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66321951
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,144 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  //TODO How is this generated?
    --- End diff --
    
    remove TODO


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66347579
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,143 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  private static final long serialVersionUID = 201462371710L;
    +  /**
    +   * The number of partitions for the default partitioner to create.
    +   */
    +  @Min(1)
    +  private int partitionCount = 1;
    +
    +  /**
    +   * This creates a partitioner which creates only one partition.
    +   */
    +  public ClonePartitioner()
    +  {
    +  }
    +
    +  /**
    +   * This constructor is used to create the partitioner from a property.
    +   * @param value A string which is an integer of the number of partitions to create
    +   */
    +  public ClonePartitioner(String value)
    +  {
    +    this(Integer.parseInt(value));
    +  }
    +
    +  /**
    +   * This creates a partitioner which creates partitonCount partitions.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public ClonePartitioner(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method sets the number of partitions for the ClonePartitioner to create.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public void setPartitionCount(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method gets the number of partitions for the ClonePartitioner to create.
    +   * @return The number of partitions to create.
    +   */
    +  public int getPartitionCount()
    +  {
    +    return partitionCount;
    +  }
    +
    +  @Override
    +  public Collection<Partition<T>> definePartitions(Collection<Partition<T>> partitions, PartitioningContext context)
    +  {
    +    final int newPartitionCount = DefaultPartition.getRequiredPartitionCount(context, this.partitionCount);
    +    logger.debug("define partitions, partitionCount current {} requested {}", partitions.size(), newPartitionCount);
    +
    +    //Get a partition
    +    DefaultPartition<T> partition = (DefaultPartition<T>)partitions.iterator().next();
    +    ArrayList<Partition<T>> newPartitions;
    +
    +    // first call to define partitions
    --- End diff --
    
    nit: `first` to `First`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r67593337
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,141 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  private static final long serialVersionUID = 201462371710L;
    +  /**
    +   * The number of partitions for the default partitioner to create.
    +   */
    +  @Min(1)
    +  private int partitionCount = 1;
    +
    +  /**
    +   * This creates a partitioner which creates only one partition.
    +   */
    +  public ClonePartitioner()
    +  {
    +  }
    +
    +  /**
    +   * This constructor is used to create the partitioner from a property.
    +   * @param value A string which is an integer of the number of partitions to create
    +   */
    +  public ClonePartitioner(String value)
    --- End diff --
    
    `value` renamed to `partitionCount` like below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66347546
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,143 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  private static final long serialVersionUID = 201462371710L;
    +  /**
    +   * The number of partitions for the default partitioner to create.
    +   */
    +  @Min(1)
    +  private int partitionCount = 1;
    +
    +  /**
    +   * This creates a partitioner which creates only one partition.
    +   */
    +  public ClonePartitioner()
    +  {
    +  }
    +
    +  /**
    +   * This constructor is used to create the partitioner from a property.
    +   * @param value A string which is an integer of the number of partitions to create
    +   */
    +  public ClonePartitioner(String value)
    +  {
    +    this(Integer.parseInt(value));
    +  }
    +
    +  /**
    +   * This creates a partitioner which creates partitonCount partitions.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public ClonePartitioner(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method sets the number of partitions for the ClonePartitioner to create.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public void setPartitionCount(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method gets the number of partitions for the ClonePartitioner to create.
    +   * @return The number of partitions to create.
    +   */
    +  public int getPartitionCount()
    +  {
    +    return partitionCount;
    +  }
    +
    +  @Override
    +  public Collection<Partition<T>> definePartitions(Collection<Partition<T>> partitions, PartitioningContext context)
    +  {
    +    final int newPartitionCount = DefaultPartition.getRequiredPartitionCount(context, this.partitionCount);
    +    logger.debug("define partitions, partitionCount current {} requested {}", partitions.size(), newPartitionCount);
    +
    +    //Get a partition
    --- End diff --
    
    nit: space after `//`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348467
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +
    --- End diff --
    
    nit: remove newline here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66322765
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +  private static Integer totalReceived = 0;
    +  private static Integer totalSent = 0;
    --- End diff --
    
    Is this variable required?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348405
  
    --- Diff: library/src/main/java/com/datatorrent/lib/partitioner/ClonePartitioner.java ---
    @@ -0,0 +1,143 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +
    +/**
    + * This is a simple partitioner, which replicates data across all partitions of an operator.
    + *
    + * @param <T> The type of the operator
    + * @since 2.0.0
    + */
    +public class ClonePartitioner<T extends Operator> implements Partitioner<T>, Serializable
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ClonePartitioner.class);
    +  
    +  private static final long serialVersionUID = 201462371710L;
    +  /**
    +   * The number of partitions for the default partitioner to create.
    +   */
    +  @Min(1)
    +  private int partitionCount = 1;
    +
    +  /**
    +   * This creates a partitioner which creates only one partition.
    +   */
    +  public ClonePartitioner()
    +  {
    +  }
    +
    +  /**
    +   * This constructor is used to create the partitioner from a property.
    +   * @param value A string which is an integer of the number of partitions to create
    +   */
    +  public ClonePartitioner(String value)
    +  {
    +    this(Integer.parseInt(value));
    +  }
    +
    +  /**
    +   * This creates a partitioner which creates partitonCount partitions.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public ClonePartitioner(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method sets the number of partitions for the ClonePartitioner to create.
    +   * @param partitionCount The number of partitions to create.
    +   */
    +  public void setPartitionCount(int partitionCount)
    +  {
    +    this.partitionCount = partitionCount;
    +  }
    +
    +  /**
    +   * This method gets the number of partitions for the ClonePartitioner to create.
    +   * @return The number of partitions to create.
    +   */
    +  public int getPartitionCount()
    +  {
    +    return partitionCount;
    +  }
    +
    +  @Override
    +  public Collection<Partition<T>> definePartitions(Collection<Partition<T>> partitions, PartitioningContext context)
    +  {
    +    final int newPartitionCount = DefaultPartition.getRequiredPartitionCount(context, this.partitionCount);
    +    logger.debug("define partitions, partitionCount current {} requested {}", partitions.size(), newPartitionCount);
    +
    +    //Get a partition
    +    DefaultPartition<T> partition = (DefaultPartition<T>)partitions.iterator().next();
    +    ArrayList<Partition<T>> newPartitions;
    +
    +    // first call to define partitions
    +    newPartitions = Lists.newArrayList();
    +
    +    Set partSet = Sets.newHashSet();
    +
    +    // Add as many partitions as we wish to have
    +    for (int partitionCounter = 0; partitionCounter < newPartitionCount; partitionCounter++) {
    +      DefaultPartition<T> partitionToAdd = new DefaultPartition<T>(partition.getPartitionedInstance());
    +      newPartitions.add(partitionToAdd);
    +      partSet.add(partitionCounter);
    +    }
    +
    +    // For every partition, accept all data from all the input ports by defining a mask that accepts the data for
    +    // every partitionId for every allocated partition and associate all partitions with every port.
    +    List<InputPort<?>> inputPortList = context.getInputPorts();
    +    if (inputPortList != null) {
    +      for (InputPort<?> port : inputPortList) {
    +        for (int i = 0; i < newPartitions.size(); i++) {
    +          newPartitions.get(i).getPartitionKeys().put(port, new PartitionKeys(i, partSet));
    +        }
    +      }
    +    }
    +
    +    logger.debug("new partition size {}", newPartitions.size());
    +    return newPartitions;
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<T>> partitions)
    +  {
    +    //Do nothing
    --- End diff --
    
    nit: space after `//`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66322272
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +  private static Integer totalReceived = 0;
    +  private static Integer totalSent = 0;
    +
    +  private static boolean matchFailed = false;
    +
    +  public static class FixedEmitter implements InputOperator
    --- End diff --
    
    may be you want to extend BaseOperator to avoid writing boiler plate code like setup/endWindow


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348752
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +
    +  private static boolean matchFailed = false;
    +
    +  public static class FixedEmitter extends BaseOperator implements InputOperator
    +  {
    +    public final DefaultOutputPort<Integer> output = new DefaultOutputPort<>();
    +
    +    private Integer emitted;
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      emitted = 0;
    +    }
    +
    +    @Override
    +    public void emitTuples()
    +    {
    +      while (emitted < countPerWindow) {
    +        output.emit(emitted++);
    +      }
    +    }
    +  }
    +
    +  public static class DummyOperator extends BaseOperator
    +  {
    +    private Integer value;
    +
    +    private transient ArrayList<Integer> valuesReceived = new ArrayList<>();
    +
    +    public DummyOperator()
    +    {
    +    }
    +
    +    public DummyOperator(Integer value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public final transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
    +    {
    +      @Override
    +      public void process(Integer tuple)
    +      {
    +        valuesReceived.add(tuple);
    +      }
    +    };
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      valuesReceived.clear();
    +    }
    +
    +    @Override
    +    public void endWindow()
    +    {
    +      if (valuesReceived.size() < countPerWindow) {
    +        matchFailed = true;
    +      } else {
    +        for (int i = 0; i < countPerWindow; i++) {
    +          if (valuesReceived.get(i) != i) {
    +            matchFailed = true;
    +          }
    +        }
    +      }
    +    }
    +
    +    public void setValue(int value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public int getValue()
    +    {
    +      return value;
    +    }
    +  }
    +
    +  @Test
    +  public void partition1Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 1, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void partition5Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>(5);
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 5, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void objectPropertyTest()
    +  {
    +    Object2String<ClonePartitioner<DummyOperator>> propertyReader = new Object2String<>();
    +    ClonePartitioner<DummyOperator> partitioner =
    +        propertyReader.fromString("com.datatorrent.lib.partitioner.ClonePartitioner:3");
    +    assertEquals(3, partitioner.getPartitionCount());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleUP()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    partitions.add(new DefaultPartition<>(dummyOperator));
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 5));
    +    assertEquals("after partition", 5, newPartitions.size());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleDown()
    --- End diff --
    
    rename to `parallelPartitionScaleDownTest`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] apex-malhar pull request #315: [APEXMALHAR-2102] Support clone partitioner

Posted by brennonyork <gi...@git.apache.org>.
Github user brennonyork commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/315#discussion_r66348808
  
    --- Diff: library/src/test/java/com/datatorrent/lib/partitioner/ClonePartitionerTest.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package com.datatorrent.lib.partitioner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.Operator.InputPort;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.Partitioner.Partition;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.api.StringCodec.Object2String;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +import static junit.framework.TestCase.assertFalse;
    +import static org.junit.Assert.assertEquals;
    +
    +public class ClonePartitionerTest
    +{
    +  private static Logger LOG = LoggerFactory.getLogger(ClonePartitionerTest.class);
    +
    +  private static int numPartitions = 20;
    +  private static Integer countPerWindow = 100;
    +
    +  private static boolean matchFailed = false;
    +
    +  public static class FixedEmitter extends BaseOperator implements InputOperator
    +  {
    +    public final DefaultOutputPort<Integer> output = new DefaultOutputPort<>();
    +
    +    private Integer emitted;
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      emitted = 0;
    +    }
    +
    +    @Override
    +    public void emitTuples()
    +    {
    +      while (emitted < countPerWindow) {
    +        output.emit(emitted++);
    +      }
    +    }
    +  }
    +
    +  public static class DummyOperator extends BaseOperator
    +  {
    +    private Integer value;
    +
    +    private transient ArrayList<Integer> valuesReceived = new ArrayList<>();
    +
    +    public DummyOperator()
    +    {
    +    }
    +
    +    public DummyOperator(Integer value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public final transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
    +    {
    +      @Override
    +      public void process(Integer tuple)
    +      {
    +        valuesReceived.add(tuple);
    +      }
    +    };
    +
    +    @Override
    +    public void beginWindow(long windowId)
    +    {
    +      valuesReceived.clear();
    +    }
    +
    +    @Override
    +    public void endWindow()
    +    {
    +      if (valuesReceived.size() < countPerWindow) {
    +        matchFailed = true;
    +      } else {
    +        for (int i = 0; i < countPerWindow; i++) {
    +          if (valuesReceived.get(i) != i) {
    +            matchFailed = true;
    +          }
    +        }
    +      }
    +    }
    +
    +    public void setValue(int value)
    +    {
    +      this.value = value;
    +    }
    +
    +    public int getValue()
    +    {
    +      return value;
    +    }
    +  }
    +
    +  @Test
    +  public void partition1Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 1, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void partition5Test()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>(5);
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    DefaultPartition<DummyOperator> defaultPartition = new DefaultPartition<>(dummyOperator);
    +    partitions.add(defaultPartition);
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 0));
    +    assertEquals("Incorrect number of partitions", 5, newPartitions.size());
    +
    +    for (Partition<DummyOperator> partition : newPartitions) {
    +      assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
    +    }
    +  }
    +
    +  @Test
    +  public void objectPropertyTest()
    +  {
    +    Object2String<ClonePartitioner<DummyOperator>> propertyReader = new Object2String<>();
    +    ClonePartitioner<DummyOperator> partitioner =
    +        propertyReader.fromString("com.datatorrent.lib.partitioner.ClonePartitioner:3");
    +    assertEquals(3, partitioner.getPartitionCount());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleUP()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +    partitions.add(new DefaultPartition<>(dummyOperator));
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 5));
    +    assertEquals("after partition", 5, newPartitions.size());
    +  }
    +
    +  @Test
    +  public void testParallelPartitionScaleDown()
    +  {
    +    DummyOperator dummyOperator = new DummyOperator(5);
    +    ClonePartitioner<DummyOperator> clonePartitioner = new ClonePartitioner<>();
    +
    +    Collection<Partition<DummyOperator>> partitions = Lists.newArrayList();
    +
    +    for (int i = 5; i-- > 0; ) {
    +      partitions.add(new DefaultPartition<>(dummyOperator));
    +    }
    +
    +    Collection<Partition<DummyOperator>> newPartitions = clonePartitioner.definePartitions(partitions,
    +        new PartitioningContextImpl(null, 1));
    +    assertEquals("after partition", 1, newPartitions.size());
    +  }
    +
    +  static class Application implements StreamingApplication
    +  {
    +    @Override
    +    public void populateDAG(DAG dag, Configuration conf)
    +    {
    +      LOG.debug("Application - PopulateDAG");
    +      FixedEmitter emitter = new FixedEmitter();
    +      DummyOperator dummy = new DummyOperator(5);
    +
    +      dag.addOperator("Emitter", emitter);
    +      dag.addOperator("Dummy", dummy);
    +
    +      ClonePartitioner<DummyOperator> partitioner = new ClonePartitioner<>();
    +      partitioner.setPartitionCount(numPartitions);
    +
    +      dag.setAttribute(dummy, Context.OperatorContext.PARTITIONER, partitioner);
    +      dag.addStream("Emitter to dummy", emitter.output, dummy.input);
    +    }
    +  }
    +
    +  @Test
    +  public void testCloningAcrossPartitions() throws IOException, Exception
    +  {
    +    try {
    +      LocalMode lma = LocalMode.newInstance();
    +      Configuration conf = new Configuration(false);
    +      lma.prepareDAG(new Application(), conf);
    +      LocalMode.Controller lc = lma.getController();
    +
    +      lc.run(20000); // runs for 10 seconds and quits
    +      assertFalse("Failed to match all values in all partitions.", matchFailed);
    +    } catch (ConstraintViolationException e) {
    +      Assert.fail("constraint violations: " + e.getConstraintViolations());
    +    }
    +  }
    +
    --- End diff --
    
    nit: remove newline


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---