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

[GitHub] apex-malhar pull request #335: [REVIEW ONLY] APEXMALHAR-1701 Deduper with Ma...

GitHub user bhupeshchawda opened a pull request:

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

    [REVIEW ONLY] APEXMALHAR-1701 Deduper with Managed state

    A Deduper based on managed state.
    Added Pojo implementation. Added unit tests

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

    $ git pull https://github.com/bhupeshchawda/apex-malhar APEXMALHAR-1701-deduper

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

    https://github.com/apache/apex-malhar/pull/335.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 #335
    
----
commit a3162323207aaf0ff92fd837d9e6787c28504041
Author: Chandni Singh <cs...@apache.org>
Date:   2016-04-29T18:43:33Z

    APEXMALHAR-1701 An abstract deduper implemenatation

commit 6ca869efa62a1ea70ec24afcc4394acbbc37a1e5
Author: bhupesh <bh...@gmail.com>
Date:   2016-07-07T10:22:34Z

    APEXMALHAR-1701: Added features to AbstractDeduper. Added Pojo implementation. Added unit 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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70587702
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicates = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<T>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  protected boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  protected transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  protected transient long uniqueEvents;
    --- End diff --
    
    uniqueEvents, duplicateEvents, expiredEvents doesn't seem to be used in subclass.. Can you make them private? Also please check if there are any other such variables which do not need access outside of this java file.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70587352
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    --- End diff --
    
    Can be replaced with <>... Other places are also there where this comment is true.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72991555
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    --- End diff --
    
    Should this be done partition-wise?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73102463
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    --- End diff --
    
    This is the base path. Partitions will add buckets depending on their operator id.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73132123
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73161100
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitUnique(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    +              processUnique(tuple);
    +            } else {
    +              processDuplicate(tuple);
    +            }
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new RuntimeException("handle idle time", e);
    +          }
    +          waitIterator.remove();
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processAuxiliary(true);
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    Preconditions.checkArgument(waitingEvents.isEmpty());
    +    asyncEvents.clear();
    +    managedState.endWindow();
    +  }
    +
    +  /**
    +   * Records a decision for use later. This is needed to ensure that the order of incoming tuples is maintained.
    +   *
    +   * @param tuple
    +   * @param d The decision for the tuple
    +   */
    +  protected void recordDecision(T tuple, Decision d)
    +  {
    +    decisions.put(tuple, d);
    +  }
    +
    +  /**
    +   * Processes tuples for which the decision (unique / duplicate / expired) has been made.
    +   * Breaks once an undecided tuple is found, as we don't want to emit out of order
    +   */
    +  protected void emitProcessedTuples()
    +  {
    +    Iterator<Entry<T, Decision>> entries = decisions.entrySet().iterator();
    +    while (entries.hasNext()) {
    +      Entry<T, Decision> td = entries.next();
    +      switch (td.getValue()) {
    +        case UNIQUE:
    +          uniqueEvents++;
    +          emitUnique(td.getKey());
    +          entries.remove();
    +          break;
    +        case DUPLICATE:
    +          duplicateEvents++;
    +          emitDuplicate(td.getKey());
    +          entries.remove();
    +          break;
    +        case EXPIRED:
    +          expiredEvents++;
    +          emitExpired(td.getKey());
    +          entries.remove();
    +          break;
    +        default:
    +          /*
    +           * Decision for this is still UNKNOWN. Tuple is still waiting for bucket to be loaded. Break.
    +           */
    +          break;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    managedState.teardown();
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long windowId)
    +  {
    +    managedState.beforeCheckpoint(windowId);
    +  }
    +
    +  @Override
    +  public void checkpointed(long windowId)
    +  {
    +    managedState.checkpointed(windowId);
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    managedState.committed(windowId);
    +  }
    +
    +  protected void emitUnique(T event)
    +  {
    +    unique.emit(event);
    +  }
    +
    +  protected void emitDuplicate(T event)
    +  {
    +    duplicate.emit(event);
    +  }
    +
    +  protected void emitExpired(T event)
    +  {
    +    expired.emit(event);
    +  }
    +
    +  /**
    +   * Checks whether output of deduper should preserve the input order
    +   */
    +  public boolean isOrderedOutput()
    +  {
    +    return preserveTupleOrder;
    +  }
    +
    +  /**
    +   * If set to true, the deduper will emit tuples in the order in which they were received. Tuples which arrived later
    +   * will wait for previous tuples to get processed and emitted. If not set, the order of tuples may change as tuples
    +   * may be emitted out of order as and when they get processed.
    +   *
    +   * @param orderedOutput
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73114470
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    --- End diff --
    
    Done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72997448
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    --- End diff --
    
    Is it possible to rename this to maintainOrder... Ordered Output looks misleading property name to me.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72998669
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  // Required properties
    +  @NotNull
    +  private String keyExpression;
    +
    +  private String timeExpression;
    +
    +  @NotNull
    +  private long bucketSpan;
    +
    +  @NotNull
    +  private long expireBefore;
    +
    +  // Optional
    +  private long referenceInstant = -1;
    +
    +  private transient Class<?> pojoClass;
    +
    +  private transient Getter<Object, Long> timeGetter;
    +
    +  private transient Getter<Object, Object> keyGetter;
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultInputPort<Object> input = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      pojoClass = context.getAttributes().get(PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getDeduperStreamCodec();
    +    }
    +  };
    +
    +  @Override
    +  protected long getTime(Object tuple)
    +  {
    +    if (timeGetter != null) {
    +      return timeGetter.get(tuple);
    +    }
    +    return System.currentTimeMillis();
    +  }
    +
    +  @Override
    +  protected Slice getKey(Object tuple)
    +  {
    +    Object key = keyGetter.get(tuple);
    +    return new Slice(key.toString().getBytes());
    +  }
    +
    +  protected StreamCodec<Object> getDeduperStreamCodec()
    +  {
    +    return new DeduperStreamCodec(keyExpression);
    +  }
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    TimeBucketAssigner timeBucketAssigner = new TimeBucketAssigner();
    +    timeBucketAssigner.setBucketSpan(Duration.standardSeconds(bucketSpan));
    +    timeBucketAssigner.setExpireBefore(Duration.standardSeconds(expireBefore));
    +    if (referenceInstant == -1) {
    +      timeBucketAssigner.setReferenceInstant(new Instant());
    --- End diff --
    
    If current time is used as reference, shouldn't this be set to referenceInstant variable as well to ensure that in case of restore of operator, timeAssigner gets same base time value?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73134200
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    --- End diff --
    
    where is asyncEvent emptied?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73103038
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    --- End diff --
    
    Intention of above approach is following:
    1. Can avoid 2 times call to getKey
    2. Can avoid some space fr storage and processing at managed state.
    
    Any other approach is also fine as long as above 2 are met.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142228
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    --- End diff --
    
    Please add description to tuple parameter.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73155029
  
    --- Diff: library/src/test/java/org/apache/apex/malhar/lib/dedup/DeduperIdempotencyTest.java ---
    @@ -0,0 +1,132 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.io.IOException;
    +import java.util.Date;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +
    +public class DeduperIdempotencyTest
    +{
    +  public static boolean testFailed = false;
    +
    +  @Test
    +  public void testApplication() throws IOException, Exception
    +  {
    +    try {
    +      LocalMode lma = LocalMode.newInstance();
    +      Configuration conf = new Configuration(false);
    +      lma.prepareDAG(new DeduperIdempotencyTestApp(), conf);
    +      LocalMode.Controller lc = lma.getController();
    +      lc.runAsync();
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73139110
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73134653
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperStreamCodec.java ---
    @@ -0,0 +1,45 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    --- End diff --
    
    Please add a javadoc comment describing how this works?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72989638
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    --- End diff --
    
    It would be great is deduper logic can be described here. That's important piece of information that user want to have.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73102615
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    --- End diff --
    
    No specific description; may be number of reasons. Will remove the description "process"


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72998388
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  // Required properties
    +  @NotNull
    +  private String keyExpression;
    +
    +  private String timeExpression;
    +
    +  @NotNull
    +  private long bucketSpan;
    +
    +  @NotNull
    +  private long expireBefore;
    +
    +  // Optional
    +  private long referenceInstant = -1;
    --- End diff --
    
    How is referenceInstant a property? Or is it a helper variable?
    From what I understood its set to timeAssigner for calculating buckets. Do we want to expose that feature to use for time shift?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73156767
  
    --- Diff: library/src/test/java/org/apache/apex/malhar/lib/dedup/DeduperIdempotencyTest.java ---
    @@ -0,0 +1,132 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.io.IOException;
    +import java.util.Date;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +
    +public class DeduperIdempotencyTest
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70589466
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  @NotNull
    +  private String keyExpression;
    +
    +  @NotNull
    +  private String timeExpression;
    +
    +  private long bucketSpan;
    --- End diff --
    
    Need to add constraint for bucketSpan..


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72996544
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    --- End diff --
    
    Why is waitingEvent condition required 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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70587314
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    --- End diff --
    
    Can be replaced with <>


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73115146
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  // Required properties
    +  @NotNull
    +  private String keyExpression;
    +
    +  private String timeExpression;
    +
    +  @NotNull
    +  private long bucketSpan;
    +
    +  @NotNull
    +  private long expireBefore;
    +
    +  // Optional
    +  private long referenceInstant = -1;
    --- End diff --
    
    It is a property for user to supply in seconds. We internally convert it to an instant.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70589676
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    --- End diff --
    
    No need t set optional=true for output ports, they're optional anyway.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142554
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    --- End diff --
    
    Description for tuple.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142337
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    --- End diff --
    
    Please add description.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73101735
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    --- End diff --
    
    Agreed. Also, will rename to ```preserveTupleOrder```


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73139012
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    --- End diff --
    
    in 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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70589402
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  @NotNull
    +  private String keyExpression;
    +
    +  @NotNull
    +  private String timeExpression;
    +
    +  private long bucketSpan;
    +
    +  private long expireBefore;
    --- End diff --
    
    If this variable is optional.. Can you explicitly assign this to 0?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142080
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitUnique(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    +              processUnique(tuple);
    +            } else {
    +              processDuplicate(tuple);
    +            }
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new RuntimeException("handle idle time", e);
    +          }
    +          waitIterator.remove();
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processAuxiliary(true);
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    Preconditions.checkArgument(waitingEvents.isEmpty());
    +    asyncEvents.clear();
    +    managedState.endWindow();
    +  }
    +
    +  /**
    +   * Records a decision for use later. This is needed to ensure that the order of incoming tuples is maintained.
    +   *
    +   * @param tuple
    +   * @param d The decision for the tuple
    +   */
    +  protected void recordDecision(T tuple, Decision d)
    +  {
    +    decisions.put(tuple, d);
    +  }
    +
    +  /**
    +   * Processes tuples for which the decision (unique / duplicate / expired) has been made.
    +   * Breaks once an undecided tuple is found, as we don't want to emit out of order
    +   */
    +  protected void emitProcessedTuples()
    +  {
    +    Iterator<Entry<T, Decision>> entries = decisions.entrySet().iterator();
    +    while (entries.hasNext()) {
    +      Entry<T, Decision> td = entries.next();
    +      switch (td.getValue()) {
    +        case UNIQUE:
    +          uniqueEvents++;
    +          emitUnique(td.getKey());
    +          entries.remove();
    +          break;
    +        case DUPLICATE:
    +          duplicateEvents++;
    +          emitDuplicate(td.getKey());
    +          entries.remove();
    +          break;
    +        case EXPIRED:
    +          expiredEvents++;
    +          emitExpired(td.getKey());
    +          entries.remove();
    +          break;
    +        default:
    +          /*
    +           * Decision for this is still UNKNOWN. Tuple is still waiting for bucket to be loaded. Break.
    +           */
    +          break;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    managedState.teardown();
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long windowId)
    +  {
    +    managedState.beforeCheckpoint(windowId);
    +  }
    +
    +  @Override
    +  public void checkpointed(long windowId)
    +  {
    +    managedState.checkpointed(windowId);
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    managedState.committed(windowId);
    +  }
    +
    +  protected void emitUnique(T event)
    +  {
    +    unique.emit(event);
    +  }
    +
    +  protected void emitDuplicate(T event)
    +  {
    +    duplicate.emit(event);
    +  }
    +
    +  protected void emitExpired(T event)
    +  {
    +    expired.emit(event);
    +  }
    +
    +  /**
    +   * Checks whether output of deduper should preserve the input order
    +   */
    +  public boolean isOrderedOutput()
    +  {
    +    return preserveTupleOrder;
    +  }
    +
    +  /**
    +   * If set to true, the deduper will emit tuples in the order in which they were received. Tuples which arrived later
    +   * will wait for previous tuples to get processed and emitted. If not set, the order of tuples may change as tuples
    +   * may be emitted out of order as and when they get processed.
    +   *
    +   * @param orderedOutput
    +   */
    +  public void setPreserveTupleOrder(boolean preserveTupleOrder)
    +  {
    +    this.preserveTupleOrder = preserveTupleOrder;
    +  }
    +
    +  public ManagedTimeUnifiedStateImpl getManagedState()
    --- End diff --
    
    removed


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73156746
  
    --- Diff: library/src/test/java/org/apache/apex/malhar/lib/dedup/DeduperPartitioningTest.java ---
    @@ -0,0 +1,171 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Date;
    +import java.util.HashMap;
    +import java.util.Random;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.common.partitioner.StatelessPartitioner;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.io.ConsoleOutputOperator;
    +
    +/**
    + * Tests whether the operator functions correctly when partitioned
    + * The partitioning in Dedup is overridden by partitioning on basis of the key in the tuple.
    + *
    + */
    +public class DeduperPartitioningTest
    +{
    +  public static final int NUM_DEDUP_PARTITIONS = 5;
    +  private static boolean testFailed = false;
    +
    +  /**
    +   * Application to test the partitioning
    +   *
    +   */
    +  public static class TestDedupApp implements StreamingApplication
    +  {
    +    @Override
    +    public void populateDAG(DAG dag, Configuration conf)
    +    {
    +      TestGenerator gen = dag.addOperator("Generator", new TestGenerator());
    +
    +      TestDeduper dedup = dag.addOperator("Deduper", new TestDeduper());
    +      dedup.setKeyExpression("id");
    +      dedup.setTimeExpression("eventTime.getTime()");
    +      dedup.setBucketSpan(60);
    +      dedup.setExpireBefore(600);
    +      
    +      ConsoleOutputOperator console = dag.addOperator("Console", new ConsoleOutputOperator());
    +      dag.addStream("Generator to Dedup", gen.output, dedup.input);
    +      dag.addStream("Dedup to Console", dedup.unique, console.input);
    +      dag.setInputPortAttribute(dedup.input, Context.PortContext.TUPLE_CLASS, TestEvent.class);
    +      dag.setOutputPortAttribute(dedup.unique, Context.PortContext.TUPLE_CLASS, TestEvent.class);
    +      dag.setAttribute(dedup, Context.OperatorContext.PARTITIONER, 
    +          new StatelessPartitioner<TimeBasedDedupOperator>(NUM_DEDUP_PARTITIONS));
    +    }
    +  }
    +
    +  public static class TestDeduper extends TimeBasedDedupOperator
    +  {
    +    int operatorId;
    +    boolean started = false;
    +    HashMap<Integer, Integer> partitionMap = Maps.newHashMap();
    +
    +    @Override
    +    public void setup(OperatorContext context)
    +    {
    +      super.setup(context);
    +      operatorId = context.getId();
    +    }
    +
    +    @Override
    +    protected void processTuple(Object tuple)
    +    {
    +      TestEvent event = (TestEvent)tuple;
    +      if (partitionMap.containsKey(event.id)) {
    +        if (partitionMap.get(event.id) != operatorId) {
    +          testFailed = true;
    +          throw new RuntimeException("Wrong tuple assignment");
    +        }
    +      } else {
    +        partitionMap.put(event.id, operatorId);
    +      }
    +    }
    +  }
    +
    +  public static class TestGenerator extends BaseOperator implements InputOperator
    +  {
    +
    +    public final transient DefaultOutputPort<TestEvent> output = new DefaultOutputPort<>();
    +    private final transient Random r = new Random();
    +
    +    @Override
    +    public void emitTuples()
    +    {
    +      TestEvent event = new TestEvent();
    +      event.id = r.nextInt(100);
    +      output.emit(event);
    +    }
    +  }
    +
    +  public static class TestEvent
    +  {
    +    private int id;
    +    private Date eventTime;
    +
    +    public TestEvent()
    +    {
    +    }
    +
    +    public int getId()
    +    {
    +      return id;
    +    }
    +
    +    public void setId(int id)
    +    {
    +      this.id = id;
    +    }
    +
    +    public Date getEventTime()
    +    {
    +      return eventTime;
    +    }
    +
    +    public void setEventTime(Date eventTime)
    +    {
    +      this.eventTime = eventTime;
    +    }
    +  }
    +
    +  /**
    +   * This test validates whether a tuple key goes to exactly one partition
    +   */
    +  @Test
    +  public void testDeduperStreamCodec()
    +  {
    +    try {
    +      LocalMode lma = LocalMode.newInstance();
    +      Configuration conf = new Configuration(false);
    +      lma.prepareDAG(new TestDedupApp(), conf);
    +      LocalMode.Controller lc = lma.getController();
    +      lc.run(10 * 1000); // runs for 10 seconds and quits
    --- End diff --
    
    done



---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142681
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitUnique(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    +              processUnique(tuple);
    +            } else {
    +              processDuplicate(tuple);
    +            }
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new RuntimeException("handle idle time", e);
    +          }
    +          waitIterator.remove();
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processAuxiliary(true);
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    Preconditions.checkArgument(waitingEvents.isEmpty());
    +    asyncEvents.clear();
    +    managedState.endWindow();
    +  }
    +
    +  /**
    +   * Records a decision for use later. This is needed to ensure that the order of incoming tuples is maintained.
    +   *
    +   * @param tuple
    +   * @param d The decision for the tuple
    +   */
    +  protected void recordDecision(T tuple, Decision d)
    +  {
    +    decisions.put(tuple, d);
    +  }
    +
    +  /**
    +   * Processes tuples for which the decision (unique / duplicate / expired) has been made.
    +   * Breaks once an undecided tuple is found, as we don't want to emit out of order
    +   */
    +  protected void emitProcessedTuples()
    +  {
    +    Iterator<Entry<T, Decision>> entries = decisions.entrySet().iterator();
    +    while (entries.hasNext()) {
    +      Entry<T, Decision> td = entries.next();
    +      switch (td.getValue()) {
    +        case UNIQUE:
    +          uniqueEvents++;
    +          emitUnique(td.getKey());
    +          entries.remove();
    +          break;
    +        case DUPLICATE:
    +          duplicateEvents++;
    +          emitDuplicate(td.getKey());
    +          entries.remove();
    +          break;
    +        case EXPIRED:
    +          expiredEvents++;
    +          emitExpired(td.getKey());
    +          entries.remove();
    +          break;
    +        default:
    +          /*
    +           * Decision for this is still UNKNOWN. Tuple is still waiting for bucket to be loaded. Break.
    +           */
    +          break;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    managedState.teardown();
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long windowId)
    +  {
    +    managedState.beforeCheckpoint(windowId);
    +  }
    +
    +  @Override
    +  public void checkpointed(long windowId)
    +  {
    +    managedState.checkpointed(windowId);
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    managedState.committed(windowId);
    +  }
    +
    +  protected void emitUnique(T event)
    +  {
    +    unique.emit(event);
    +  }
    +
    +  protected void emitDuplicate(T event)
    +  {
    +    duplicate.emit(event);
    +  }
    +
    +  protected void emitExpired(T event)
    +  {
    +    expired.emit(event);
    +  }
    +
    +  /**
    +   * Checks whether output of deduper should preserve the input order
    +   */
    +  public boolean isOrderedOutput()
    +  {
    +    return preserveTupleOrder;
    +  }
    +
    +  /**
    +   * If set to true, the deduper will emit tuples in the order in which they were received. Tuples which arrived later
    +   * will wait for previous tuples to get processed and emitted. If not set, the order of tuples may change as tuples
    +   * may be emitted out of order as and when they get processed.
    +   *
    +   * @param orderedOutput
    --- End diff --
    
    Incorrect param in javadoc. Also add description to it.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142273
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    --- End diff --
    
    Please add descriptions to parameters.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73115138
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    --- End diff --
    
    I don't see value present in asyncEvents being used anywhere.. Can you put NULL Object instead?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r71270785
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicates = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<T>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  protected boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  protected transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  protected transient long uniqueEvents;
    --- End diff --
    
    Done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73102835
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    --- End diff --
    
    We are checking whether we can directly proceed to process a tuple and emit it. If we don't need to preserve order, then we can proceeed. Also even if we need to preserve the order, and there is no other event that is waiting; then we can proceed.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73104101
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    --- End diff --
    
    Within a single call to handleIdleTime, can you please take care of a single waitingEvent instead of take care of all the events in a single handleIdleTime.
    In endWindow, this is fine.. But in handleIdleTime, you should allow engine to control amount of time handleIdleTime is called.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73141245
  
    --- Diff: library/src/test/java/org/apache/apex/malhar/lib/dedup/DeduperIdempotencyTest.java ---
    @@ -0,0 +1,132 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.io.IOException;
    +import java.util.Date;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +
    +public class DeduperIdempotencyTest
    --- End diff --
    
    Knit: Can you rename this to DeduperOrderTest ? I don't this this test idempotency really.. :)


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73160450
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73112564
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    --- End diff --
    
    where is tuples from asyncEvent removed?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73115095
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  // Required properties
    +  @NotNull
    +  private String keyExpression;
    +
    +  private String timeExpression;
    +
    +  @NotNull
    +  private long bucketSpan;
    +
    +  @NotNull
    +  private long expireBefore;
    +
    +  // Optional
    +  private long referenceInstant = -1;
    +
    +  private transient Class<?> pojoClass;
    +
    +  private transient Getter<Object, Long> timeGetter;
    +
    +  private transient Getter<Object, Object> keyGetter;
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultInputPort<Object> input = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      pojoClass = context.getAttributes().get(PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getDeduperStreamCodec();
    +    }
    +  };
    +
    +  @Override
    +  protected long getTime(Object tuple)
    +  {
    +    if (timeGetter != null) {
    +      return timeGetter.get(tuple);
    +    }
    +    return System.currentTimeMillis();
    +  }
    +
    +  @Override
    +  protected Slice getKey(Object tuple)
    +  {
    +    Object key = keyGetter.get(tuple);
    +    return new Slice(key.toString().getBytes());
    +  }
    +
    +  protected StreamCodec<Object> getDeduperStreamCodec()
    +  {
    +    return new DeduperStreamCodec(keyExpression);
    +  }
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    TimeBucketAssigner timeBucketAssigner = new TimeBucketAssigner();
    +    timeBucketAssigner.setBucketSpan(Duration.standardSeconds(bucketSpan));
    +    timeBucketAssigner.setExpireBefore(Duration.standardSeconds(expireBefore));
    +    if (referenceInstant == -1) {
    +      timeBucketAssigner.setReferenceInstant(new Instant());
    --- End diff --
    
    Have set the default in the declaration. Will remove the if statement.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73102939
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    --- End diff --
    
    I understand you don't care about value for putting in managed state. Maybe en null Object can be put... For eg..
    Have a static inner class;
      private static class NullObject
      {
      }
    
      private static final NullObject NULL = new NullObject();
    
    And use NULL as value while putting into managed state.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73160693
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitUnique(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    +              processUnique(tuple);
    +            } else {
    +              processDuplicate(tuple);
    +            }
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new RuntimeException("handle idle time", e);
    +          }
    +          waitIterator.remove();
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processAuxiliary(true);
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    Preconditions.checkArgument(waitingEvents.isEmpty());
    +    asyncEvents.clear();
    +    managedState.endWindow();
    +  }
    +
    +  /**
    +   * Records a decision for use later. This is needed to ensure that the order of incoming tuples is maintained.
    +   *
    +   * @param tuple
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73138996
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperStreamCodec.java ---
    @@ -0,0 +1,45 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +public class DeduperStreamCodec extends KryoSerializableStreamCodec<Object>
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72993201
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    --- End diff --
    
    Better describe the exception?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70588730
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  @NotNull
    +  private String keyExpression;
    +
    +  @NotNull
    +  private String timeExpression;
    +
    +  private long bucketSpan;
    +
    +  private long expireBefore;
    +
    +  private long referenceInstant;
    +
    +  private transient Class<?> pojoClass;
    +
    +  private transient Getter<Object, Long> timeGetter;
    +
    +  private transient Getter<Object, Object> keyGetter;
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultInputPort<Object> input = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      pojoClass = context.getAttributes().get(PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  @Override
    +  protected long getTime(Object tuple)
    +  {
    +    return timeGetter.get(tuple);
    +  }
    +
    +  @Override
    +  protected Slice getKey(Object tuple)
    +  {
    +    Object key = keyGetter.get(tuple);
    +    return new Slice(key.toString().getBytes());
    +  }
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    TimeBucketAssigner tba = new TimeBucketAssigner();
    +    tba.setBucketSpan(Duration.standardSeconds(bucketSpan));
    +    tba.setExpireBefore(Duration.standardSeconds(expireBefore));
    +    tba.setReferenceInstant(new Instant(referenceInstant));
    +    managedState.setTimeBucketAssigner(tba);
    +    super.setup(context);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    timeGetter = PojoUtils.createGetter(pojoClass, timeExpression, Long.class);
    +    keyGetter = PojoUtils.createGetter(pojoClass, keyExpression, Object.class);
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  public String getKeyExpression()
    +  {
    +    return keyExpression;
    +  }
    +
    +  public void setKeyExpression(String keyExpression)
    --- End diff --
    
    Can you please add javadoc for all the setter methods? This is often the most important information that users want to see while using any operator.
    
    Also while adding javadoc can you please add UI metatags?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70587509
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicates = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<T>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  protected boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    --- End diff --
    
    Here the new value is assigned and made @NotNull... I don't think this make sense.. What is the purpose of @NotNull 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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73132107
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73132546
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    --- End diff --
    
    Yes, adding value was just temporary as I did not have anything else there and null was not accepted. I'll use NullObject as suggested.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72989366
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    --- End diff --
    
    javadoc is missing for this. Pease describe how the deduper works. 


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70589147
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  @NotNull
    +  private String keyExpression;
    +
    +  @NotNull
    +  private String timeExpression;
    +
    +  private long bucketSpan;
    --- End diff --
    
    A minor knit: Can you categorize these properties together using javadoc sections: mandatory, optional, helper.. This way user gets to know what needs to be set.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r71270664
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    --- End diff --
    
    anonymous class


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72994134
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    --- End diff --
    
    I suggest we make this true by default and call it out explicitly of performance penalty of doing this.
    The reason for default true is then one can be sure that default deduper is idempotent which usually a assumption while targeting end-to-end exactly once.
    
    Choice should be given on making it performant at the cost of idempotency. Not the other way round.



---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73160639
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73160437
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r71270656
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    --- End diff --
    
    Will change it


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73139258
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitUnique(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    +              processUnique(tuple);
    +            } else {
    +              processDuplicate(tuple);
    +            }
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new RuntimeException("handle idle time", e);
    +          }
    +          waitIterator.remove();
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processAuxiliary(true);
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    Preconditions.checkArgument(waitingEvents.isEmpty());
    +    asyncEvents.clear();
    +    managedState.endWindow();
    +  }
    +
    +  /**
    +   * Records a decision for use later. This is needed to ensure that the order of incoming tuples is maintained.
    +   *
    +   * @param tuple
    +   * @param d The decision for the tuple
    +   */
    +  protected void recordDecision(T tuple, Decision d)
    +  {
    +    decisions.put(tuple, d);
    +  }
    +
    +  /**
    +   * Processes tuples for which the decision (unique / duplicate / expired) has been made.
    +   * Breaks once an undecided tuple is found, as we don't want to emit out of order
    +   */
    +  protected void emitProcessedTuples()
    +  {
    +    Iterator<Entry<T, Decision>> entries = decisions.entrySet().iterator();
    +    while (entries.hasNext()) {
    +      Entry<T, Decision> td = entries.next();
    +      switch (td.getValue()) {
    +        case UNIQUE:
    +          uniqueEvents++;
    +          emitUnique(td.getKey());
    +          entries.remove();
    +          break;
    +        case DUPLICATE:
    +          duplicateEvents++;
    +          emitDuplicate(td.getKey());
    +          entries.remove();
    +          break;
    +        case EXPIRED:
    +          expiredEvents++;
    +          emitExpired(td.getKey());
    +          entries.remove();
    +          break;
    +        default:
    +          /*
    +           * Decision for this is still UNKNOWN. Tuple is still waiting for bucket to be loaded. Break.
    +           */
    +          break;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    managedState.teardown();
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long windowId)
    +  {
    +    managedState.beforeCheckpoint(windowId);
    +  }
    +
    +  @Override
    +  public void checkpointed(long windowId)
    +  {
    +    managedState.checkpointed(windowId);
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    managedState.committed(windowId);
    +  }
    +
    +  protected void emitUnique(T event)
    +  {
    +    unique.emit(event);
    +  }
    +
    +  protected void emitDuplicate(T event)
    +  {
    +    duplicate.emit(event);
    +  }
    +
    +  protected void emitExpired(T event)
    +  {
    +    expired.emit(event);
    +  }
    +
    +  /**
    +   * Checks whether output of deduper should preserve the input order
    +   */
    +  public boolean isOrderedOutput()
    +  {
    +    return preserveTupleOrder;
    +  }
    +
    +  /**
    +   * If set to true, the deduper will emit tuples in the order in which they were received. Tuples which arrived later
    +   * will wait for previous tuples to get processed and emitted. If not set, the order of tuples may change as tuples
    +   * may be emitted out of order as and when they get processed.
    +   *
    +   * @param orderedOutput
    +   */
    +  public void setPreserveTupleOrder(boolean preserveTupleOrder)
    +  {
    +    this.preserveTupleOrder = preserveTupleOrder;
    +  }
    +
    +  public ManagedTimeUnifiedStateImpl getManagedState()
    --- End diff --
    
    Why is this getter 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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70588006
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicates = new DefaultOutputPort<T>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<T>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  protected boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  protected transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  protected transient long uniqueEvents;
    +  @AutoMetric
    +  protected transient long duplicateEvents;
    +  @AutoMetric
    +  protected transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param bucket
    --- End diff --
    
    Please correct javadoc for parameters.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72997177
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    --- End diff --
    
    knit: emitUnique... Just to to be similar on all places.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73160600
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    --- End diff --
    
    done


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70589489
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    +{
    +
    +  @NotNull
    +  private String keyExpression;
    +
    +  @NotNull
    +  private String timeExpression;
    +
    +  private long bucketSpan;
    +
    +  private long expireBefore;
    +
    +  private long referenceInstant;
    --- End diff --
    
    Need to add constraint for referenceInstant


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73141070
  
    --- Diff: library/src/test/java/org/apache/apex/malhar/lib/dedup/DeduperIdempotencyTest.java ---
    @@ -0,0 +1,132 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.io.IOException;
    +import java.util.Date;
    +
    +import javax.validation.ConstraintViolationException;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +
    +public class DeduperIdempotencyTest
    +{
    +  public static boolean testFailed = false;
    +
    +  @Test
    +  public void testApplication() throws IOException, Exception
    +  {
    +    try {
    +      LocalMode lma = LocalMode.newInstance();
    +      Configuration conf = new Configuration(false);
    +      lma.prepareDAG(new DeduperIdempotencyTestApp(), conf);
    +      LocalMode.Controller lc = lma.getController();
    +      lc.runAsync();
    --- End diff --
    
    Can you run the test in async mode of Controller using method similar to:
    com.datatorrent.lib.io.fs.FileSplitterBaseTest.testSplitterInApp
    
    This gives a deterministic way of verification the cases.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73141100
  
    --- Diff: library/src/test/java/org/apache/apex/malhar/lib/dedup/DeduperPartitioningTest.java ---
    @@ -0,0 +1,171 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Date;
    +import java.util.HashMap;
    +import java.util.Random;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.api.StreamingApplication;
    +import com.datatorrent.common.partitioner.StatelessPartitioner;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.io.ConsoleOutputOperator;
    +
    +/**
    + * Tests whether the operator functions correctly when partitioned
    + * The partitioning in Dedup is overridden by partitioning on basis of the key in the tuple.
    + *
    + */
    +public class DeduperPartitioningTest
    +{
    +  public static final int NUM_DEDUP_PARTITIONS = 5;
    +  private static boolean testFailed = false;
    +
    +  /**
    +   * Application to test the partitioning
    +   *
    +   */
    +  public static class TestDedupApp implements StreamingApplication
    +  {
    +    @Override
    +    public void populateDAG(DAG dag, Configuration conf)
    +    {
    +      TestGenerator gen = dag.addOperator("Generator", new TestGenerator());
    +
    +      TestDeduper dedup = dag.addOperator("Deduper", new TestDeduper());
    +      dedup.setKeyExpression("id");
    +      dedup.setTimeExpression("eventTime.getTime()");
    +      dedup.setBucketSpan(60);
    +      dedup.setExpireBefore(600);
    +      
    +      ConsoleOutputOperator console = dag.addOperator("Console", new ConsoleOutputOperator());
    +      dag.addStream("Generator to Dedup", gen.output, dedup.input);
    +      dag.addStream("Dedup to Console", dedup.unique, console.input);
    +      dag.setInputPortAttribute(dedup.input, Context.PortContext.TUPLE_CLASS, TestEvent.class);
    +      dag.setOutputPortAttribute(dedup.unique, Context.PortContext.TUPLE_CLASS, TestEvent.class);
    +      dag.setAttribute(dedup, Context.OperatorContext.PARTITIONER, 
    +          new StatelessPartitioner<TimeBasedDedupOperator>(NUM_DEDUP_PARTITIONS));
    +    }
    +  }
    +
    +  public static class TestDeduper extends TimeBasedDedupOperator
    +  {
    +    int operatorId;
    +    boolean started = false;
    +    HashMap<Integer, Integer> partitionMap = Maps.newHashMap();
    +
    +    @Override
    +    public void setup(OperatorContext context)
    +    {
    +      super.setup(context);
    +      operatorId = context.getId();
    +    }
    +
    +    @Override
    +    protected void processTuple(Object tuple)
    +    {
    +      TestEvent event = (TestEvent)tuple;
    +      if (partitionMap.containsKey(event.id)) {
    +        if (partitionMap.get(event.id) != operatorId) {
    +          testFailed = true;
    +          throw new RuntimeException("Wrong tuple assignment");
    +        }
    +      } else {
    +        partitionMap.put(event.id, operatorId);
    +      }
    +    }
    +  }
    +
    +  public static class TestGenerator extends BaseOperator implements InputOperator
    +  {
    +
    +    public final transient DefaultOutputPort<TestEvent> output = new DefaultOutputPort<>();
    +    private final transient Random r = new Random();
    +
    +    @Override
    +    public void emitTuples()
    +    {
    +      TestEvent event = new TestEvent();
    +      event.id = r.nextInt(100);
    +      output.emit(event);
    +    }
    +  }
    +
    +  public static class TestEvent
    +  {
    +    private int id;
    +    private Date eventTime;
    +
    +    public TestEvent()
    +    {
    +    }
    +
    +    public int getId()
    +    {
    +      return id;
    +    }
    +
    +    public void setId(int id)
    +    {
    +      this.id = id;
    +    }
    +
    +    public Date getEventTime()
    +    {
    +      return eventTime;
    +    }
    +
    +    public void setEventTime(Date eventTime)
    +    {
    +      this.eventTime = eventTime;
    +    }
    +  }
    +
    +  /**
    +   * This test validates whether a tuple key goes to exactly one partition
    +   */
    +  @Test
    +  public void testDeduperStreamCodec()
    +  {
    +    try {
    +      LocalMode lma = LocalMode.newInstance();
    +      Configuration conf = new Configuration(false);
    +      lma.prepareDAG(new TestDedupApp(), conf);
    +      LocalMode.Controller lc = lma.getController();
    +      lc.run(10 * 1000); // runs for 10 seconds and quits
    --- End diff --
    
    Can you run the test in async mode of Controller using method similar to:
    com.datatorrent.lib.io.fs.FileSplitterBaseTest.testSplitterInApp
    
    This gives a deterministic way of verification the cases.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73134312
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperStreamCodec.java ---
    @@ -0,0 +1,45 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +public class DeduperStreamCodec extends KryoSerializableStreamCodec<Object>
    --- End diff --
    
    Can you make this Evolving?


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73132603
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,462 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained
    +   */
    +  private boolean orderedOutput = false;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #orderedOutput} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (orderedOutput) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException("process", e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (orderedOutput) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!orderedOutput || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (orderedOutput && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitOutput(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (orderedOutput) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    --- End diff --
    
    Used in the if condition


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70588532
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    --- End diff --
    
    Please make this class Evolving.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70588401
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    --- End diff --
    
    Can you please fix variable and method access in AbstractDeduper.. Most of them doesn't seem to required protected.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r72989231
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperTimeBasedPOJOImpl.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +import com.datatorrent.netlet.util.Slice;
    +
    +@Evolving
    +public class DeduperTimeBasedPOJOImpl extends AbstractDeduper<Object> implements ActivationListener<Context>
    --- End diff --
    
    Can this be renamed to "TimeBasedDeduper".. I don't think Impl or POJO is 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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r70589909
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,463 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
    --- End diff --
    
    Can we call this port "unique" instead of "output"... To me it makes more sense to call 3 ports of deduper as "unique", "duplicates" & "expired"


---
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 #335: APEXMALHAR-1701 Deduper with Managed state

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73142614
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/AbstractDeduper.java ---
    @@ -0,0 +1,459 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.BucketedState;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeUnifiedStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.TimeBucketAssigner;
    +import org.apache.hadoop.classification.InterfaceStability.Evolving;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.AutoMetric;
    +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.Operator;
    +import com.datatorrent.api.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Abstract class which allows de-duplicating incoming tuples based on a configured key.
    + * Also supports expiry mechanism based on a configurable expiry period configured using {@link TimeBucketAssigner}
    + * in {@link ManagedTimeUnifiedStateImpl}
    + * Following steps are used in identifying the state of a particular tuple:
    + * 1. Check if the time of the tuple falls in an expired bucket. If so, the tuple is expired
    + * 2. If the tuple is a valid event, it is checked in the store whether the same key already exists in the
    + * time bucket identified by the event time. If, so, the tuple is a duplicate.
    + * 3. Otherwise the tuple is a unique tuple.
    + *
    + * @param <T> type of events
    + */
    +@Evolving
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public abstract class AbstractDeduper<T>
    +    implements Operator, Operator.IdleTimeHandler, ActivationListener<Context>, Operator.CheckpointNotificationListener
    +{
    +  /**
    +   * The input port on which events are received.
    +   */
    +  public final transient DefaultInputPort<T> input = new DefaultInputPort<T>()
    +  {
    +    @Override
    +    public final void process(T tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * The output port on which deduped events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> unique = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which duplicate events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> duplicate = new DefaultOutputPort<>();
    +
    +  /**
    +   * The output port on which expired events are emitted.
    +   */
    +  public final transient DefaultOutputPort<T> expired = new DefaultOutputPort<>();
    +
    +  /**
    +   * Whether or not the order of tuples be maintained.
    +   * Making this "true" might entail some cost in performance, but makes the operator idempotent.
    +   */
    +  private boolean preserveTupleOrder = true;
    +
    +  @NotNull
    +  protected final ManagedTimeUnifiedStateImpl managedState = new ManagedTimeUnifiedStateImpl();
    +
    +  /**
    +   * Map to hold the result of a tuple processing (unique, duplicate, expired or error) until previous
    +   * tuples get processed. This is used only when {@link #preserveTupleOrder} is true.
    +   */
    +  private transient Map<T, Decision> decisions;
    +  private transient long sleepMillis;
    +  private transient Map<T, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private transient Map<Slice, Long> asyncEvents = Maps.newLinkedHashMap();
    +
    +  // Metrics
    +  @AutoMetric
    +  private transient long uniqueEvents;
    +  @AutoMetric
    +  private transient long duplicateEvents;
    +  @AutoMetric
    +  private transient long expiredEvents;
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    FileAccessFSImpl fAccessImpl = new TFileImpl.DTFileImpl();
    +    fAccessImpl.setBasePath(context.getValue(DAG.APPLICATION_PATH) + "/bucket_data");
    +    managedState.setFileAccess(fAccessImpl);
    +    managedState.setup(context);
    +
    +    if (preserveTupleOrder) {
    +      decisions = Maps.newLinkedHashMap();
    +    }
    +  }
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    // Reset Metrics
    +    uniqueEvents = 0;
    +    duplicateEvents = 0;
    +    expiredEvents = 0;
    +
    +    managedState.beginWindow(l);
    +  }
    +
    +  protected abstract Slice getKey(T event);
    +
    +  protected abstract long getTime(T event);
    +
    +  /**
    +   * Processes an incoming tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processTuple(T tuple)
    +  {
    +
    +    long time = getTime(tuple);
    +    Future<Slice> valFuture = managedState.getAsync(time, getKey(tuple));
    +
    +    if (valFuture.isDone()) {
    +      try {
    +        processEvent(tuple, valFuture.get());
    +      } catch (InterruptedException | ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, valFuture);
    +    }
    +  }
    +
    +  /**
    +   * Processes a looked-up event
    +   *
    +   * @param tuple
    +   * @param value
    +   */
    +  protected void processEvent(T tuple, Slice value)
    +  {
    +    if (value == BucketedState.EXPIRED) {
    +      processInvalid(tuple);
    +      return;
    +    }
    +    processValid(tuple, value);
    +  }
    +
    +  /**
    +   * Processes a tuple which is waiting for the lookup to return.
    +   *
    +   * @param tuple The tuple which needs to wait
    +   * @param future The future object which will ultimately return the lookup result
    +   */
    +  protected void processWaitingEvent(T tuple, Future<Slice> future)
    +  {
    +    waitingEvents.put(tuple, future);
    +    if (preserveTupleOrder) {
    +      recordDecision(tuple, Decision.UNKNOWN);
    +    }
    +  }
    +
    +  /**
    +   * Processes a valid (non-expired) tuple. This tuple may be a unique or a duplicate.
    +   *
    +   * @param tuple
    +   *          The tuple to be processed
    +   * @param value
    +   *          Looked up key of the tuple
    +   */
    +  protected void processValid(T tuple, Slice value)
    +  {
    +    if (!preserveTupleOrder || waitingEvents.isEmpty()) {
    +      if (value == null) {
    +        managedState.put(getTime(tuple), getKey(tuple), getKey(tuple));
    +        processUnique(tuple);
    +      } else {
    +        processDuplicate(tuple);
    +      }
    +    } else {
    +      processWaitingEvent(tuple, Futures.immediateFuture(value));
    +    }
    +  }
    +
    +  /**
    +   * Processes invalid tuples.
    +   *
    +   * @param tuple
    +   */
    +  protected void processInvalid(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.EXPIRED);
    +    } else {
    +      processExpired(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes an expired tuple
    +   *
    +   * @param tuple
    +   */
    +  protected void processExpired(T tuple)
    +  {
    +    expiredEvents++;
    +    emitExpired(tuple);
    +  }
    +
    +  /**
    +   * Processes the duplicate tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a duplicate
    +   */
    +  protected void processDuplicate(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.DUPLICATE);
    +    } else {
    +      duplicateEvents++;
    +      emitDuplicate(tuple);
    +    }
    +  }
    +
    +  /**
    +   * Processes the unique tuple.
    +   *
    +   * @param tuple
    +   *          The tuple which is a unique
    +   */
    +  protected void processUnique(T tuple)
    +  {
    +    if (preserveTupleOrder && !decisions.isEmpty()) {
    +      recordDecision(tuple, Decision.UNIQUE);
    +    } else {
    +      uniqueEvents++;
    +      emitUnique(tuple);
    +    }
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    processAuxiliary(false);
    +  }
    +
    +  /**
    +   * Does any auxiliary processing in the idle time of the operator.
    +   * Processes any tuples which are waiting for the lookup to return.
    +   *
    +   * @param finalize Whether or not to wait for future to return
    +   */
    +  protected void processAuxiliary(boolean finalize)
    +  {
    +    if (waitingEvents.size() > 0) {
    +      Iterator<Map.Entry<T, Future<Slice>>> waitIterator = waitingEvents.entrySet().iterator();
    +      while (waitIterator.hasNext()) {
    +        Map.Entry<T, Future<Slice>> waitingEvent = waitIterator.next();
    +        T tuple = waitingEvent.getKey();
    +        Slice tupleKey = getKey(tuple);
    +        long tupleTime = getTime(tuple);
    +        Future<Slice> future = waitingEvent.getValue();
    +        if (future.isDone() || finalize ) {
    +          try {
    +            if (future.get() == null && asyncEvents.get(tupleKey) == null) {
    +              managedState.put(tupleTime, tupleKey, tupleKey);
    +              asyncEvents.put(tupleKey, tupleTime);
    +              processUnique(tuple);
    +            } else {
    +              processDuplicate(tuple);
    +            }
    +          } catch (InterruptedException | ExecutionException e) {
    +            throw new RuntimeException("handle idle time", e);
    +          }
    +          waitIterator.remove();
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processAuxiliary(true);
    +    if (preserveTupleOrder) {
    +      emitProcessedTuples();
    +    }
    +    Preconditions.checkArgument(waitingEvents.isEmpty());
    +    asyncEvents.clear();
    +    managedState.endWindow();
    +  }
    +
    +  /**
    +   * Records a decision for use later. This is needed to ensure that the order of incoming tuples is maintained.
    +   *
    +   * @param tuple
    --- End diff --
    
    Missing description.


---
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 #335: [Review Only] APEXMALHAR-1701 Deduper with Ma...

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

    https://github.com/apache/apex-malhar/pull/335#discussion_r73141988
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/dedup/DeduperStreamCodec.java ---
    @@ -0,0 +1,45 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.dedup;
    +
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    --- End diff --
    
    done


---
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.
---