[ 
https://issues.apache.org/jira/browse/APEXMALHAR-1701?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15403862#comment-15403862
 ] 

ASF GitHub Bot commented on APEXMALHAR-1701:
--------------------------------------------

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.


> Deduper : create a deduper backed by Managed State
> --------------------------------------------------
>
>                 Key: APEXMALHAR-1701
>                 URL: https://issues.apache.org/jira/browse/APEXMALHAR-1701
>             Project: Apache Apex Malhar
>          Issue Type: Task
>          Components: algorithms
>            Reporter: Chandni Singh
>            Assignee: Chandni Singh
>
> Need a de-deduplicator operator that is based on Managed State.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to