Return-Path: X-Original-To: apmail-apex-dev-archive@minotaur.apache.org Delivered-To: apmail-apex-dev-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1AA811822F for ; Thu, 26 Nov 2015 05:44:46 +0000 (UTC) Received: (qmail 21798 invoked by uid 500); 26 Nov 2015 05:44:46 -0000 Delivered-To: apmail-apex-dev-archive@apex.apache.org Received: (qmail 21742 invoked by uid 500); 26 Nov 2015 05:44:46 -0000 Mailing-List: contact dev-help@apex.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@apex.incubator.apache.org Delivered-To: mailing list dev@apex.incubator.apache.org Received: (qmail 21731 invoked by uid 99); 26 Nov 2015 05:44:45 -0000 Received: from Unknown (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 Nov 2015 05:44:45 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 45292C14AA for ; Thu, 26 Nov 2015 05:44:45 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.427 X-Spam-Level: X-Spam-Status: No, score=0.427 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.554, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id XukXa4UmD0_L for ; Thu, 26 Nov 2015 05:44:40 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with SMTP id AABF721231 for ; Thu, 26 Nov 2015 05:44:38 +0000 (UTC) Received: (qmail 21684 invoked by uid 99); 26 Nov 2015 05:44:37 -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; Thu, 26 Nov 2015 05:44:37 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BB8AFE2C10; Thu, 26 Nov 2015 05:44:37 +0000 (UTC) From: gauravgopi123 To: dev@apex.incubator.apache.org Reply-To: dev@apex.incubator.apache.org References: In-Reply-To: Subject: [GitHub] incubator-apex-malhar pull request: SPOI-4520 Implemented Inmemory... Content-Type: text/plain Message-Id: <20151126054437.BB8AFE2C10@git1-us-west.apache.org> Date: Thu, 26 Nov 2015 05:44:37 +0000 (UTC) Github user gauravgopi123 commented on a diff in the pull request: https://github.com/apache/incubator-apex-malhar/pull/113#discussion_r45945847 --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractJoinOperator.java --- @@ -0,0 +1,343 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package com.datatorrent.lib.join; + +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceStability.Evolving; + +import com.datatorrent.api.Context; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.Operator; +import com.datatorrent.api.annotation.InputPortFieldAnnotation; +import com.datatorrent.common.util.BaseOperator; + +/** + *

+ * This is the base implementation of join operator. Operator receives tuples from multiple streams, + * applies the join operation based on constraint and emit the joined value. + * Subclasses should provide implementation to createOutputTuple,copyValue, getKeyValue, getTime methods. + * + * Properties:
+ * expiryTime: Expiry time for stored tuples
+ * includeFieldStr: List of comma separated fields to be added to the output tuple. + * Ex: Field1,Field2;Field3,Field4
+ * keyFields: List of comma separated key field for both the streams. Ex: Field1,Field2
+ * timeFields: List of comma separated time field for both the streams. Ex: Field1,Field2
+ * bucketSpanInMillis: Span of each bucket in milliseconds.
+ * strategy: Type of join operation. Default type is inner join
+ *
+ *

+ * + * @displayName Abstract Join Operator + * @tags join + */ +@Evolving +public abstract class AbstractJoinOperator extends BaseOperator implements Operator.CheckpointListener +{ + public final transient DefaultOutputPort> outputPort = new DefaultOutputPort<>(); + protected String[] timeFields; + + protected String[][] includeFields; + // Fields to compare from both the streams + protected String[] keys; + // Strategy of Join operation, by default the option is inner join + protected JoinStrategy strategy = JoinStrategy.INNER_JOIN; + // This represents whether the processing tuple is from left port or not + protected Boolean isLeft; + @InputPortFieldAnnotation(optional = true) + public transient DefaultInputPort input1 = new DefaultInputPort() + { + @Override + public void process(T tuple) + { + isLeft = true; + processTuple(tuple); + } + }; + @InputPortFieldAnnotation(optional = true) + public transient DefaultInputPort input2 = new DefaultInputPort() + { + @Override + public void process(T tuple) + { + isLeft = false; + processTuple(tuple); + } + }; + // Stores for each of the input port + private BackupStore[] store = (BackupStore[])Array.newInstance(BackupStore.class, 2); + private String includeFieldStr; + private String keyFieldStr; + private String timeFieldStr; + + @Override + public void setup(Context.OperatorContext context) + { + if (store[0] == null) { + throw new RuntimeException("Left Store is Empty"); + } + if (store[1] == null) { + throw new RuntimeException("Right Store is Empty"); + } + // Checks whether the strategy is outer join and set it to store + boolean isOuter = strategy.equals(JoinStrategy.LEFT_OUTER_JOIN) || strategy.equals(JoinStrategy.OUTER_JOIN); + store[0].isOuterJoin(isOuter); + isOuter = strategy.equals(JoinStrategy.RIGHT_OUTER_JOIN) || strategy.equals(JoinStrategy.OUTER_JOIN); + store[1].isOuterJoin(isOuter); + // Setup the stores + store[0].setup(); + store[1].setup(); + + populateFields(); + } + + /** + * Create the event with the given tuple. If it successfully inserted it into the store + * then it does the join operation + * + * @param tuple Tuple to process + */ + protected void processTuple(T tuple) + { + int idx = 0; + if (!isLeft) { + idx = 1; + } + TimeEvent t = createEvent(tuple); + if (store[idx].put(t)) { + join(t, isLeft); + } + } + + private void populateFields() + { + populateIncludeFields(); + populateKeyFields(); + if (timeFieldStr != null) { + populateTimeFields(); + } + } + + /** + * Populate the fields from the includeFiledStr + */ + private void populateIncludeFields() + { + includeFields = new String[2][]; + String[] portFields = includeFieldStr.split(";"); + for (int i = 0; i < portFields.length; i++) { + includeFields[i] = portFields[i].split(","); + } + } + + /** + * Get the tuples from another store based on join constraint and key + * + * @param tuple input + * @param isLeft whether the given tuple is from first port or not + */ + private void join(TimeEvent tuple, Boolean isLeft) + { + // Get the valid tuples from the store based on key + // If the tuple is null means the join type is outer and return unmatched tuples from store. + Object value; + if (isLeft) { + if (tuple != null) { + value = store[1].getValidTuples(tuple); + } else { + value = store[1].getUnMatchedTuples(); + } + } else { + if (tuple != null) { + value = store[0].getValidTuples(tuple); + } else { + value = store[0].getUnMatchedTuples(); + } + } + // Join the input tuple with the joined tuples + if (value != null) { + ArrayList joinedValues = (ArrayList)value; + List result = new ArrayList<>(); + for (TimeEvent joinedValue : joinedValues) { + T output = createOutputTuple(); + Object tupleValue = null; + if (tuple != null) { + tupleValue = tuple.getValue(); + } + copyValue(output, tupleValue, isLeft); + copyValue(output, joinedValue.getValue(), !isLeft); + result.add(output); + joinedValue.setMatch(true); + } + if (tuple != null) { + tuple.setMatch(true); + } + if (result.size() != 0) { + outputPort.emit(result); + } + } + } + + // Emit the unmatched tuples, if the strategy is outer join + @Override + public void endWindow() + { + if (strategy.equals(JoinStrategy.LEFT_OUTER_JOIN) || strategy.equals(JoinStrategy.OUTER_JOIN)) { + join(null, false); + } + if (strategy.equals(JoinStrategy.RIGHT_OUTER_JOIN) || strategy.equals(JoinStrategy.OUTER_JOIN)) { + join(null, true); + } + store[0].endWindow(); + store[1].endWindow(); + } + + @Override + public void checkpointed(long windowId) + { + store[0].checkpointed(windowId); + store[1].checkpointed(windowId); + } + + @Override + public void committed(long windowId) + { + store[0].committed(windowId); + store[1].committed(windowId); + } + + /** + * Convert the given tuple to event + * + * @param tuple Given tuple to convert into event + * @return event + */ + protected TimeEvent createEvent(Object tuple) + { + int idx = 0; + if (!isLeft) { + idx = 1; + } + if (timeFields != null) { + return new TimeEventImpl(getKeyValue(keys[idx], tuple), (Long)getTime(timeFields[idx], tuple), tuple); + } else { + return new TimeEventImpl(getKeyValue(keys[idx], tuple), Calendar.getInstance().getTimeInMillis(), tuple); + } + } + + public void populateKeyFields() + { --- End diff -- why is this public? only usage i see is in populateFields() --- 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. ---