Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id C1BFA200B51 for ; Mon, 1 Aug 2016 17:22:44 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id BEC27160A6C; Mon, 1 Aug 2016 15:22:44 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id DCA91160A66 for ; Mon, 1 Aug 2016 17:22:43 +0200 (CEST) Received: (qmail 11421 invoked by uid 500); 1 Aug 2016 15:22:43 -0000 Mailing-List: contact dev-help@apex.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@apex.apache.org Delivered-To: mailing list dev@apex.apache.org Received: (qmail 11410 invoked by uid 99); 1 Aug 2016 15:22:42 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Aug 2016 15:22:42 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 678481A12EA for ; Mon, 1 Aug 2016 15:22:42 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -5.446 X-Spam-Level: X-Spam-Status: No, score=-5.446 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-1.426] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id Y1GKR9x2npIZ for ; Mon, 1 Aug 2016 15:22:40 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 212C55F306 for ; Mon, 1 Aug 2016 15:22:38 +0000 (UTC) Received: (qmail 11403 invoked by uid 99); 1 Aug 2016 15:22:38 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Aug 2016 15:22:38 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2D18DE058E; Mon, 1 Aug 2016 15:22:38 +0000 (UTC) From: chinmaykolhatkar To: dev@apex.incubator.apache.org Reply-To: dev@apex.incubator.apache.org References: In-Reply-To: Subject: [GitHub] apex-malhar pull request #335: [Review Only] APEXMALHAR-1701 Deduper with Ma... Content-Type: text/plain Message-Id: <20160801152238.2D18DE058E@git1-us-west.apache.org> Date: Mon, 1 Aug 2016 15:22:38 +0000 (UTC) archived-at: Mon, 01 Aug 2016 15:22:44 -0000 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 type of events + */ +@Evolving +@OperatorAnnotation(checkpointableWithinAppWindow = false) +public abstract class AbstractDeduper + implements Operator, Operator.IdleTimeHandler, ActivationListener, Operator.CheckpointNotificationListener +{ + /** + * The input port on which events are received. + */ + public final transient DefaultInputPort input = new DefaultInputPort() + { + @Override + public final void process(T tuple) + { + processTuple(tuple); + } + }; + + /** + * The output port on which deduped events are emitted. + */ + public final transient DefaultOutputPort unique = new DefaultOutputPort<>(); + + /** + * The output port on which duplicate events are emitted. + */ + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort duplicate = new DefaultOutputPort<>(); + + /** + * The output port on which expired events are emitted. + */ + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort 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 decisions; + private transient long sleepMillis; + private transient Map> waitingEvents = Maps.newLinkedHashMap(); + private transient Map 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 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 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. ---