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

[GitHub] apex-malhar pull request #330: Initial cut of Inner Join operator for REVIEW...

GitHub user chaithu14 opened a pull request:

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

    Initial cut of Inner Join operator for REVIEW only

    

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

    $ git pull https://github.com/chaithu14/incubator-apex-malhar APEXMALHAR-2100-InnerJoin

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

    https://github.com/apache/apex-malhar/pull/330.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 #330
    
----
commit 6987dce47a5789e8de578ae3ee77c2cf6507fcc0
Author: Chaitanya <ch...@datatorrent.com>
Date:   2016-07-05T12:14:46Z

    Initial cut of Inner Join operator for REVIEW only

----


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289851
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.join.managed.ManagedSpillableComplexComponent;
    +import com.datatorrent.lib.join.managed.ManagedTimeStateMultiMap;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, isStream2KeyPrimary());
    +  }
    +
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    ((ManagedTimeStateMultiMap)store).put(key, tuple,timeBucket);
    +    joinStream(key,tuple,isStream1Data);
    +  }
    +
    +  @Override
    +  protected void joinStream(K key, T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    Future<Slice> future = ((ManagedTimeStateMultiMap)store).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(key,tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  private void joinStream(K key, T tuple, boolean isStream1Data, Slice valueSlice)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    List<T> value = null;
    +    if (((ManagedTimeStateMultiMap)store).isKeyContainsMultiValue()) {
    +      value = (List<T>)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice != null && valueSlice.length != 0 && valueSlice.buffer != null) {
    +        value = new ArrayList<>();
    +        value.add((T)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice));
    +      }
    +    }
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    +      for (T joinedValue : value) {
    +        T result = isStream1Data ? joinTuples(Arrays.asList(tuple, joinedValue)) :
    +            joinTuples(Arrays.asList(joinedValue, tuple));
    +        if (result != null) {
    +          emitTuple(result);
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    --- 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72087960
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], oc));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    +   */
    +  @Override
    +  public Object mergeTuples(Object tuple1, Object tuple2)
    +  {
    +    Object o;
    +    try {
    +      o = outputClass.newInstance();
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[0].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple1));
    +      }
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[1].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple2));
    +      }
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return o;
    +  }
    +
    +  /**
    +   * Emit the given tuple through the outputPort
    +   * @param tuple given tuple
    +   */
    +  @Override
    +  public void emitTuple(Object tuple)
    +  {
    +    outputPort.emit(tuple);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    generateSettersAndGetters();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    time += timeIncrement;
    +  }
    +
    +  /**
    +   * Returns the streamcodec for the streams
    +   * @param isStream1data Specifies whether the codec needs for stream1 or stream2.
    +   * @return the object of InnerJoinStreamCodec
    +   */
    +  public StreamCodec<Object> getInnerJoinStreamCodec(boolean isStream1data)
    +  {
    +    if (isStream1data) {
    +      return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[0]);
    +    }
    +    return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[1]);
    +  }
    +
    +  private class FieldObjectMap
    +  {
    +    public Class<?> inputClass;
    +    public PojoUtils.Getter keyGet;
    +    public PojoUtils.Getter timeFieldGet;
    +    public Map<PojoUtils.Getter,PojoUtils.Setter> fieldMap;
    --- End diff --
    
    This fieldMap seems to be outside of the scope of any specific inputClass. Can you please move this variable outside of this class and treat it as a seperate entity. Its a mapping of getter (from either stream's POJO) to setter to output POJO but there has to be a single mapping.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72383366
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    --- End diff --
    
    Please have more descriptive names for tc, ic, oc etc.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844275
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/InnerJoinStreamCodec.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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 com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Stream codec based on keyExpression for POJO Inner Join Operator.
    + *
    + * @Since 3.5.0
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

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


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73309683
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,363 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isKeyContainsMultiValue)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isKeyContainsMultiValue;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +      return null;
    +    }
    +    if (isKeyContainsMultiValue) {
    +      return (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }
    +    value = new ArrayList<>();
    +    value.add((V)streamCodec.fromByteArray(valueSlice));
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    --- End diff --
    
    This is not accessed anywhere else. Please make this private.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72222501
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    --- End diff --
    
    I agree with expiryTime to be mandatory but not with making bucketSpanTime mandatory. This quite advanced property and should have a default value.
    
    Also why do we have different expiry time for separate streams?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289826
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    joinStream(tuple,isStream1Data, valuestore.get(key));
    +  }
    +
    +  /**
    +   * Merge the given tuple and list of values.
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @param value list of tuples
    +   */
    +  protected void joinStream(T tuple, boolean isStream1Data, List<T> value)
    +  {
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    --- End diff --
    
    If the tuples did not receive any matching tuples within expiry time then those will remove from store after expiry. 
    No. we are not emitting on a separate port.
    This is not straight forward because the tuples are storing as byte[]. I will add this feature when we will work on outer join support.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844231
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    --- End diff --
    
    Ok


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72215213
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    --- End diff --
    
    When would this return false? Are we not allowing duplicates?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72109540
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    joinStream(tuple,isStream1Data, valuestore.get(key));
    +  }
    +
    +  /**
    +   * Merge the given tuple and list of values.
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @param value list of tuples
    +   */
    +  protected void joinStream(T tuple, boolean isStream1Data, List<T> value)
    +  {
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    --- End diff --
    
    This condition (!= null) will take care of the fact that if opposite stream has not got matching tuple, then ignore this join call.
    What happens to the tuples for which opposite stream did not receive any matching tuples within expiry time.
    Should those tuples be emitted on a separate port? I understand this is the case for Outer Joins, but this is just a question in regards to data lineage. The non-matching tuples will get lost in the pipeline, IMO we should provide a way out for them.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72107490
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getStream1ExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream1ExpiryTime()));
    +    }
    +    if (getStream2ExpiryTime() != null) {
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream2ExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    --- End diff --
    
    Can we process only one element in one handleIdleTime callback?
    This is to make sure that processing inside handleIdleTime does not block windows.
    handleIdleTime is called over and over again 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72218447
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    --- End diff --
    
    I will suggest making bucketSpanTime as well as the expiryTimes to be made mandatory. Reason is that if a user forgot to set one of these, there could be issues. 


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

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


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72224850
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    --- End diff --
    
    time should be System.currentTimeMillis() ?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73141502
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    --- End diff --
    
    Yes. It can be. If the timeField of the stream is not in milliseconds then the user has to override the extractTime() method and convert the field into milliseconds.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73322874
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    joinStream(tuple,isStream1Data, valuestore.get(key));
    +  }
    +
    +  /**
    +   * Merge the given tuple and list of values.
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @param value list of tuples
    +   */
    +  protected void joinStream(T tuple, boolean isStream1Data, List<T> value)
    +  {
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    --- End diff --
    
    Agreed. I suggest we make this as assumption for this version of Inner Join. Later on when we have Outer join, we can make inner join as a special case of outer join which is when we can take care of this. Lets create a Jira for addition of this functionality in inner join when this PR gets merged.
    
    Let me know if you think otherwise.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844285
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289992
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    --- End diff --
    
    Yes. 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73281797
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    --- End diff --
    
    Ok. Will make expiryTime as mandatory. 
    I thought of different use case and this is invalid in streaming scenario. 
    I will make single expiry time for both the streams.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73141485
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    --- End diff --
    
    Yes. If there are multiple key fields from each stream then the primary key field has to specify in keyFields and the other key fields has to take care in mergeTuples().


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73310059
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,363 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isKeyContainsMultiValue)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isKeyContainsMultiValue;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +      return null;
    +    }
    +    if (isKeyContainsMultiValue) {
    +      return (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }
    +    value = new ArrayList<>();
    +    value.add((V)streamCodec.fromByteArray(valueSlice));
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    +  {
    +    return isKeyContainsMultiValue;
    +  }
    +
    +  public void setKeyContainsMultiValue(boolean keyContainsMultiValue)
    +  {
    +    isKeyContainsMultiValue = keyContainsMultiValue;
    +  }
    +
    +  public long getTimeBucket()
    +  {
    +    return timeBucket;
    +  }
    +
    +  public void setTimeBucket(long timeBucket)
    +  {
    +    this.timeBucket = timeBucket;
    +  }
    +
    +  public StreamCodec getStreamCodec()
    +  {
    +    return streamCodec;
    +  }
    +
    +  public void setStreamCodec(StreamCodec streamCodec)
    +  {
    +    this.streamCodec = streamCodec;
    +  }
    +
    +  public class CompositeFuture implements Future<List>
    +  {
    +    public Future<Slice> slice;
    +
    +    public CompositeFuture(Future<Slice> slice)
    +    {
    +      this.slice = slice;
    +    }
    +
    +    @Override
    +    public boolean cancel(boolean b)
    +    {
    +      return slice.cancel(b);
    +    }
    +
    +    @Override
    +    public boolean isCancelled()
    +    {
    +      return slice.isCancelled();
    +    }
    +
    +    @Override
    +    public boolean isDone()
    +    {
    +      return slice.isDone();
    +    }
    +
    +    /**
    +     * Converts the single element into the list.
    +     * @return
    --- End diff --
    
    Missing description for return.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72401329
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    --- End diff --
    
    Shouldn't this if block be outside the nested if? Perhaps, just after you populate ```valueSlice``` from ```store.getAsync()```


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73141506
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    --- End diff --
    
    ms. Will add it into java docs.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72194687
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/ManagedSpillableComplexComponent.java ---
    @@ -0,0 +1,122 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * SpillableComplexComponent for Join Operator
    + */
    +public class ManagedSpillableComplexComponent implements SpillableComplexComponent
    --- End diff --
    
    This is used only in AbstractManagedStateInnerJoinOperator.createStore().
    
    In the interest of keeping the number of files less, can the content of method be directly used in createStore?
    It just seem to create a new object of type ManagedTimeStateMultiValue.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73282037
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    --- End diff --
    
    Yes. Time is initialized to  System.currentTimeMillis()  in declaration and incremented at every 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289947
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    --- 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72108125
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getStream1ExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream1ExpiryTime()));
    +    }
    +    if (getStream2ExpiryTime() != null) {
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream2ExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    +      } catch (InterruptedException ie) {
    +        throw new RuntimeException(ie);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waitingEvents.
    +   */
    +  private void processWaitEvents()
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      try {
    +        JoinEvent<K,T> event = waitingEvent.getKey();
    +        joinStream(event.value,event.isStream1Data,waitingEvent.getValue().get());
    --- End diff --
    
    If this Future request has not completed, the Future.get() call will hold the operator thread.
    Suggesting to do following:
    if (waitingEvent.getValue().isDone()) {
      // Process join
    }
    else {
      // To make sure idempotency is maintained join needs to happen in the same order
      return;
    }


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

Posted by chaithu14 <gi...@git.apache.org>.
GitHub user chaithu14 reopened a pull request:

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

    APEXMALHAR-2100 Implementation of Inner Join operator

    

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

    $ git pull https://github.com/chaithu14/incubator-apex-malhar APEXMALHAR-2100-InnerJoin

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

    https://github.com/apache/apex-malhar/pull/330.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 #330
    
----
commit 363287a2cca2a0a5dbb9096b4d971719e71d5342
Author: Chaitanya <ch...@datatorrent.com>
Date:   2016-08-17T07:08:23Z

    APEXMALHAR-2100 Implementation of Inner Join operator

----


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73320461
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    --- End diff --
    
    I'm not sure if that's the best behaviour... Because its really uncertain that what stream should overwrite on what. Any other approach would be possible for this?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72087402
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    --- End diff --
    
    Can you please add a check for whether includeFields[i] is present in both give input and output 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72402661
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +        return null;
    +      }
    +      value = new ArrayList<>();
    +      value.add((V)streamCodec.fromByteArray(valueSlice));
    +    }
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    +  {
    +    return isKeyContainsMultiValue;
    +  }
    +
    +  public void setKeyContainsMultiValue(boolean keyContainsMultiValue)
    +  {
    +    isKeyContainsMultiValue = keyContainsMultiValue;
    +  }
    +
    +  public long getTimeBucket()
    +  {
    +    return timeBucket;
    +  }
    +
    +  public void setTimeBucket(long timeBucket)
    +  {
    +    this.timeBucket = timeBucket;
    +  }
    +
    +  public StreamCodec getStreamCodec()
    +  {
    +    return streamCodec;
    +  }
    +
    +  public void setStreamCodec(StreamCodec streamCodec)
    +  {
    +    this.streamCodec = streamCodec;
    +  }
    +
    +  public class CompositeFuture implements Future<List>
    --- End diff --
    
    Instead of having this entire class, is it possible that you can just create a call called ```getAsync()```? You can then return the slice as a ```Future``` and let the caller handle whether the contents of the slice is a single value or multi values.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

Posted by chaithu14 <gi...@git.apache.org>.
GitHub user chaithu14 reopened a pull request:

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

    APEXMALHAR-2100 Implementation of Inner Join operator

    

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

    $ git pull https://github.com/chaithu14/incubator-apex-malhar APEXMALHAR-2100-InnerJoin

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

    https://github.com/apache/apex-malhar/pull/330.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 #330
    
----
commit 755aadae8814887431cd66b55bd1e03591afb5c6
Author: Chaitanya <ch...@datatorrent.com>
Date:   2016-08-17T10:22:24Z

    APEXMALHAR-2100 Implementation of Inner Join operator

----


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

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


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70042536
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,187 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +
    +  private Long expiryTime;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +  private boolean isStream1KeyPrimary = false;
    +  private boolean isStream2KeyPrimary = false;
    +
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +
    +  protected void processTuple(T tuple, boolean isStream1Data)
    --- End diff --
    
    Instead of boolean , can you pass the store directly?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72211592
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    --- End diff --
    
    Does this mean Field1 is from stream1 and Field2 is from stream2? What if there are multiple key fields from each table?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70789740
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,187 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +
    +  private Long expiryTime;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +  private boolean isStream1KeyPrimary = false;
    +  private boolean isStream2KeyPrimary = false;
    +
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +
    +  protected void processTuple(T tuple, boolean isStream1Data)
    --- End diff --
    
    No. While joining, we have to look into opposite store.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289857
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getStream1ExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream1ExpiryTime()));
    +    }
    +    if (getStream2ExpiryTime() != null) {
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream2ExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    +      } catch (InterruptedException ie) {
    +        throw new RuntimeException(ie);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waitingEvents.
    +   */
    +  private void processWaitEvents()
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      try {
    +        JoinEvent<K,T> event = waitingEvent.getKey();
    +        joinStream(event.value,event.isStream1Data,waitingEvent.getValue().get());
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297876
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    --- End diff --
    
    If LOG is not used, please remove 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72087671
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    --- End diff --
    
    If the includeField is present in both input stream tuples and their values are different, what governs which value will be copied to output POJO?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844252
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +    stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents(false);
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waiting events
    +   * @param finalize finalize Whether or not to wait for future to return
    +   */
    +  private void processWaitEvents(boolean finalize)
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      Future<List> future = waitingEvent.getValue();
    +      if (future.isDone() || finalize) {
    +        try {
    +          JoinEvent<K,T> event = waitingEvent.getKey();
    +          joinStream(event.value,event.isStream1Data,future.get());
    +        } catch (InterruptedException | ExecutionException e) {
    +          throw new RuntimeException("end window", e);
    +        }
    +        waitIterator.remove();
    +        if (!finalize) {
    +          break;
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844368
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,363 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isKeyContainsMultiValue)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isKeyContainsMultiValue;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +      return null;
    +    }
    +    if (isKeyContainsMultiValue) {
    +      return (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }
    +    value = new ArrayList<>();
    +    value.add((V)streamCodec.fromByteArray(valueSlice));
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297584
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], outputField));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    +   */
    +  @Override
    +  public Object mergeTuples(Object tuple1, Object tuple2)
    +  {
    +    Object o;
    +    try {
    +      o = outputClass.newInstance();
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[0].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple1));
    +      }
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[1].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple2));
    +      }
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return o;
    +  }
    +
    +  /**
    +   * Emit the given tuple through the outputPort
    +   * @param tuple given tuple
    +   */
    +  @Override
    +  public void emitTuple(Object tuple)
    +  {
    +    outputPort.emit(tuple);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    generateSettersAndGetters();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    time += timeIncrement;
    +  }
    +
    +  /**
    +   * Returns the streamcodec for the streams
    +   * @param isStream1data Specifies whether the codec needs for stream1 or stream2.
    +   * @return the object of InnerJoinStreamCodec
    +   */
    +  public StreamCodec<Object> getInnerJoinStreamCodec(boolean isStream1data)
    --- End diff --
    
    This can be made protected. Public is not 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73310472
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    --- End diff --
    
    Apologies... Yes you're right.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844320
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], outputField));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844300
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    --- 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72212006
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    --- End diff --
    
    In seconds? ms?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72220885
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getStream1ExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream1ExpiryTime()));
    +    }
    +    if (getStream2ExpiryTime() != null) {
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream2ExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    +      } catch (InterruptedException ie) {
    +        throw new RuntimeException(ie);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    sleepMillis = context.getValue(Context.OperatorContext.SPIN_MILLIS);
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waitingEvents.
    +   */
    +  private void processWaitEvents()
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      try {
    +        JoinEvent<K,T> event = waitingEvent.getKey();
    +        joinStream(event.value,event.isStream1Data,waitingEvent.getValue().get());
    --- End diff --
    
    Agree with Chinmay. In addition to that, you will need to have a flag which tells whether this is being called from handleIdleTime() or endWindow(). If it is endWindow(), you will need to call Future.get() until you have the value; else you won't be able to close the window. 


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72224628
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    --- End diff --
    
    This is already done in the declaration.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289967
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    --- End diff --
    
    In this case, it copies the stream2 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844311
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], outputField));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73145124
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    joinStream(tuple,isStream1Data, valuestore.get(key));
    +  }
    +
    +  /**
    +   * Merge the given tuple and list of values.
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @param value list of tuples
    +   */
    +  protected void joinStream(T tuple, boolean isStream1Data, List<T> value)
    +  {
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    --- End diff --
    
    If the tuples did not receive any matching tuples within expiry time then those will remove from store after expiry. 
    No. we are not emitting on a separate port.
    This is not straight forward because the tuples are storing as byte[]. I will add this feature when we will work on outer join support.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72088473
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    --- End diff --
    
    outputClass can be transient. Its assigned in output port's setup anyway. No need to save 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289845
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,265 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getStream1ExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream1ExpiryTime()));
    +    }
    +    if (getStream2ExpiryTime() != null) {
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getStream2ExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

Posted by chaithu14 <gi...@git.apache.org>.
GitHub user chaithu14 reopened a pull request:

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

    APEXMALHAR-2100 Implementation of Inner Join operator

    

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

    $ git pull https://github.com/chaithu14/incubator-apex-malhar APEXMALHAR-2100-InnerJoin

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

    https://github.com/apache/apex-malhar/pull/330.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 #330
    
----
commit 22e165c2cc8710c4a5135d64cb6a684cf0866c07
Author: Chaitanya <ch...@datatorrent.com>
Date:   2016-08-10T14:01:28Z

    APEXMALHAR-2100 Implementation of Inner Join operator

----


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72107558
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.join.managed.ManagedSpillableComplexComponent;
    +import com.datatorrent.lib.join.managed.ManagedTimeStateMultiMap;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, isStream2KeyPrimary());
    +  }
    +
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    ((ManagedTimeStateMultiMap)store).put(key, tuple,timeBucket);
    +    joinStream(key,tuple,isStream1Data);
    +  }
    +
    +  @Override
    +  protected void joinStream(K key, T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    Future<Slice> future = ((ManagedTimeStateMultiMap)store).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(key,tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  private void joinStream(K key, T tuple, boolean isStream1Data, Slice valueSlice)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    List<T> value = null;
    +    if (((ManagedTimeStateMultiMap)store).isKeyContainsMultiValue()) {
    +      value = (List<T>)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice != null && valueSlice.length != 0 && valueSlice.buffer != null) {
    +        value = new ArrayList<>();
    +        value.add((T)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice));
    +      }
    +    }
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    +      for (T joinedValue : value) {
    +        T result = isStream1Data ? joinTuples(Arrays.asList(tuple, joinedValue)) :
    +            joinTuples(Arrays.asList(joinedValue, tuple));
    +        if (result != null) {
    +          emitTuple(result);
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    --- End diff --
    
    If platform adds delay,, can this be 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297340
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    --- End diff --
    
    Description for what is returned.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72087056
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    --- End diff --
    
    same for timeFields.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297470
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], outputField));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    --- End diff --
    
    return description missing.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73320298
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    --- End diff --
    
    Yes... that's sufficient.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289986
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    --- 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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70789750
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.join.managed.ManagedSpillableComplexComponent;
    +import com.datatorrent.lib.join.managed.ManagedTimeStateMultiMap;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, isStream2KeyPrimary());
    +  }
    +
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    ((ManagedTimeStateMultiMap)store).put(key, tuple,timeBucket);
    +    joinStream(key,tuple,isStream1Data);
    +  }
    +
    +  @Override
    +  protected void joinStream(K key, T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    Future<Slice> future = ((ManagedTimeStateMultiMap)store).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(key,tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  private void joinStream(K key, T tuple, boolean isStream1Data, Slice valueSlice)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    List<T> value = null;
    +    if (((ManagedTimeStateMultiMap)store).isKeyContainsMultiValue()) {
    +      value = (List<T>)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice != null && valueSlice.length != 0 && valueSlice.buffer != null) {
    +        value = new ArrayList<>();
    +        value.add((T)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice));
    +      }
    +    }
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    +      for (T joinedValue : value) {
    +        T result = isStream1Data ? joinTuples(Arrays.asList(tuple, joinedValue)) :
    +            joinTuples(Arrays.asList(joinedValue, tuple));
    +        if (result != null) {
    +          emitTuple(result);
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    --- End diff --
    
    Platform adds the delay.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73282003
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +        return null;
    +      }
    +      value = new ArrayList<>();
    +      value.add((V)streamCodec.fromByteArray(valueSlice));
    +    }
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    +  {
    +    return isKeyContainsMultiValue;
    +  }
    +
    +  public void setKeyContainsMultiValue(boolean keyContainsMultiValue)
    +  {
    +    isKeyContainsMultiValue = keyContainsMultiValue;
    +  }
    +
    +  public long getTimeBucket()
    +  {
    +    return timeBucket;
    +  }
    +
    +  public void setTimeBucket(long timeBucket)
    +  {
    +    this.timeBucket = timeBucket;
    +  }
    +
    +  public StreamCodec getStreamCodec()
    +  {
    +    return streamCodec;
    +  }
    +
    +  public void setStreamCodec(StreamCodec streamCodec)
    +  {
    +    this.streamCodec = streamCodec;
    +  }
    +
    +  public class CompositeFuture implements Future<List>
    --- End diff --
    
    To make it consistent between the get() and getAsync(), created the compositeFuture.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72088749
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    --- End diff --
    
    Any specific reason why time is non-transient. I believe this variable is used to extract time from the tuple. But as this is non-transient, it'll restore back from the state even after crash. Is this expected behaviour?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289869
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/ManagedSpillableComplexComponent.java ---
    @@ -0,0 +1,122 @@
    +/**
    + * 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 org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * SpillableComplexComponent for Join Operator
    + */
    +public class ManagedSpillableComplexComponent implements SpillableComplexComponent
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289963
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    --- End diff --
    
    There is no api to check whether the field is present in Class. 
    If the includeField not exists then it will return an exception. Is this not sufficient?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70043528
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.join.managed.ManagedSpillableComplexComponent;
    +import com.datatorrent.lib.join.managed.ManagedTimeStateMultiMap;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getExpiryTime() != null) {
    +      stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +      stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    }
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, isStream2KeyPrimary());
    +  }
    +
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    ((ManagedTimeStateMultiMap)store).put(key, tuple,timeBucket);
    +    joinStream(key,tuple,isStream1Data);
    +  }
    +
    +  @Override
    +  protected void joinStream(K key, T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    Future<Slice> future = ((ManagedTimeStateMultiMap)store).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(key,tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  private void joinStream(K key, T tuple, boolean isStream1Data, Slice valueSlice)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    List<T> value = null;
    +    if (((ManagedTimeStateMultiMap)store).isKeyContainsMultiValue()) {
    +      value = (List<T>)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice != null && valueSlice.length != 0 && valueSlice.buffer != null) {
    +        value = new ArrayList<>();
    +        value.add((T)((ManagedTimeStateMultiMap)store).getStreamCodec().fromByteArray(valueSlice));
    +      }
    +    }
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    +      for (T joinedValue : value) {
    +        T result = isStream1Data ? joinTuples(Arrays.asList(tuple, joinedValue)) :
    +            joinTuples(Arrays.asList(joinedValue, tuple));
    +        if (result != null) {
    +          emitTuple(result);
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents();
    +    } else {
    +      /* nothing to do here, so sleep for a while to avoid busy loop */
    +      try {
    +        Thread.sleep(sleepMillis);
    --- End diff --
    
    is it needed, or platform adds the delay? 


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

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


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70789744
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,187 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +
    +  private Long expiryTime;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +  private boolean isStream1KeyPrimary = false;
    +  private boolean isStream2KeyPrimary = false;
    +
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    store.put(key, tuple);
    +    joinStream(key,tuple,isStream1Data);
    +  }
    +
    +  protected void joinStream(K key, T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    List<T> value = store.get(key);
    +
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    +      for (T joinedValue : value) {
    +        T result = isStream1Data ? joinTuples(Arrays.asList(tuple, joinedValue)) :
    --- End diff --
    
    List is not needed. Will update.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73320743
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], oc));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    +   */
    +  @Override
    +  public Object mergeTuples(Object tuple1, Object tuple2)
    +  {
    +    Object o;
    +    try {
    +      o = outputClass.newInstance();
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[0].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple1));
    +      }
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[1].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple2));
    +      }
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return o;
    +  }
    +
    +  /**
    +   * Emit the given tuple through the outputPort
    +   * @param tuple given tuple
    +   */
    +  @Override
    +  public void emitTuple(Object tuple)
    +  {
    +    outputPort.emit(tuple);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    generateSettersAndGetters();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    time += timeIncrement;
    +  }
    +
    +  /**
    +   * Returns the streamcodec for the streams
    +   * @param isStream1data Specifies whether the codec needs for stream1 or stream2.
    +   * @return the object of InnerJoinStreamCodec
    +   */
    +  public StreamCodec<Object> getInnerJoinStreamCodec(boolean isStream1data)
    +  {
    +    if (isStream1data) {
    +      return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[0]);
    +    }
    +    return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[1]);
    +  }
    +
    +  private class FieldObjectMap
    +  {
    +    public Class<?> inputClass;
    +    public PojoUtils.Getter keyGet;
    +    public PojoUtils.Getter timeFieldGet;
    +    public Map<PojoUtils.Getter,PojoUtils.Setter> fieldMap;
    --- End diff --
    
    Why do you want to relate it? Shoudn't there be a one-to-one mapping of output POJO fields to input POJO fields from either one of the stream?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70789733
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.join.managed.ManagedSpillableComplexComponent;
    +import com.datatorrent.lib.join.managed.ManagedTimeStateMultiMap;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getExpiryTime() != null) {
    --- End diff --
    
    Yes. It's possible. I added different parameters for both the streams.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73309976
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,363 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isKeyContainsMultiValue)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isKeyContainsMultiValue;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +      return null;
    +    }
    +    if (isKeyContainsMultiValue) {
    +      return (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }
    +    value = new ArrayList<>();
    +    value.add((V)streamCodec.fromByteArray(valueSlice));
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    --- End diff --
    
    isKeyContainsMultiValue  &  setKeyContainsMultiValue are unused. Please remove 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297388
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    --- End diff --
    
    This can be made private.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289976
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], oc));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    +   */
    +  @Override
    +  public Object mergeTuples(Object tuple1, Object tuple2)
    +  {
    +    Object o;
    +    try {
    +      o = outputClass.newInstance();
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[0].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple1));
    +      }
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[1].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple2));
    +      }
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return o;
    +  }
    +
    +  /**
    +   * Emit the given tuple through the outputPort
    +   * @param tuple given tuple
    +   */
    +  @Override
    +  public void emitTuple(Object tuple)
    +  {
    +    outputPort.emit(tuple);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    generateSettersAndGetters();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    time += timeIncrement;
    +  }
    +
    +  /**
    +   * Returns the streamcodec for the streams
    +   * @param isStream1data Specifies whether the codec needs for stream1 or stream2.
    +   * @return the object of InnerJoinStreamCodec
    +   */
    +  public StreamCodec<Object> getInnerJoinStreamCodec(boolean isStream1data)
    +  {
    +    if (isStream1data) {
    +      return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[0]);
    +    }
    +    return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[1]);
    +  }
    +
    +  private class FieldObjectMap
    +  {
    +    public Class<?> inputClass;
    +    public PojoUtils.Getter keyGet;
    +    public PojoUtils.Getter timeFieldGet;
    +    public Map<PojoUtils.Getter,PojoUtils.Setter> fieldMap;
    --- End diff --
    
    If it has a single mapping then it's difficult to check whether the getter is relates to which of the stream.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844228
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73298635
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +    stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents(false);
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waiting events
    +   * @param finalize finalize Whether or not to wait for future to return
    +   */
    +  private void processWaitEvents(boolean finalize)
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      Future<List> future = waitingEvent.getValue();
    +      if (future.isDone() || finalize) {
    +        try {
    +          JoinEvent<K,T> event = waitingEvent.getKey();
    +          joinStream(event.value,event.isStream1Data,future.get());
    +        } catch (InterruptedException | ExecutionException e) {
    +          throw new RuntimeException("end window", e);
    +        }
    +        waitIterator.remove();
    +        if (!finalize) {
    +          break;
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processWaitEvents(true);
    +    stream1Store.endWindow();
    +    stream2Store.endWindow();
    +    super.endWindow();
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    stream1Store.teardown();
    +    stream2Store.teardown();
    +    super.teardown();
    +  }
    +
    +  /**
    +   * Return the number of buckets
    +   * @return the noOfBuckets
    +   */
    +  public int getNoOfBuckets()
    +  {
    +    return noOfBuckets;
    +  }
    +
    +  /**
    +   * Set the number of buckets required for managed state
    +   * @param noOfBuckets noOfBuckets
    +   */
    +  public void setNoOfBuckets(int noOfBuckets)
    +  {
    +    this.noOfBuckets = noOfBuckets;
    +  }
    +
    +  /**
    +   * Return the bucketSpanTime
    +   * @return the bucketSpanTime
    +   */
    +  public Long getBucketSpanTime()
    +  {
    +    return bucketSpanTime;
    +  }
    +
    +  /**
    +   * Sets the length of the time bucket required for managed state.
    +   * @param bucketSpanTime given bucketSpanTime
    +   */
    +  public void setBucketSpanTime(Long bucketSpanTime)
    +  {
    +    this.bucketSpanTime = bucketSpanTime;
    +  }
    +
    +  public static class JoinEvent<K,T>
    +  {
    +    public K key;
    +    public T value;
    +    public boolean isStream1Data;
    +
    +    public JoinEvent(K key, T value, boolean isStream1Data)
    +    {
    +      this.key = key;
    +      this.value = value;
    +      this.isStream1Data = isStream1Data;
    +    }
    +  }
    +
    +  public class ManagedSpillableComplexComponent implements SpillableComplexComponent
    --- End diff --
    
    Do you need this complete class to be implemented? 
    Can you not do:
    new ManagedTimeStateMultiValue<K,V>(store, isPrimaryKey);
    new ManagedTimeStateMultiValue<K,V>();
    
    Wherever 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72211647
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    --- End diff --
    
    Does this need to be of type java.util.Date?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289943
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    --- End diff --
    
    No. activate of operator is called after operator setup.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70049062
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/managed/ManagedTimeStateMultiMap.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.Future;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public class ManagedTimeStateMultiMap<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiMap()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiMap(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +        return null;
    +      }
    +      value = new ArrayList<>();
    +      value.add((V)streamCodec.fromByteArray(valueSlice));
    +    }
    +    return  value;
    +  }
    +
    +  public Future<Slice> getAsync(@Nullable K k)
    --- End diff --
    
    ideally it should return Future<List<V>>. can you write a composible future to implement 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297901
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    --- End diff --
    
    LOG seems to be unsed, please remove 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73141529
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    --- End diff --
    
    If the tuple is already expired tuple then in this case will return false. put() will allow the duplicates.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73321794
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +    stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents(false);
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waiting events
    +   * @param finalize finalize Whether or not to wait for future to return
    +   */
    +  private void processWaitEvents(boolean finalize)
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      Future<List> future = waitingEvent.getValue();
    +      if (future.isDone() || finalize) {
    +        try {
    +          JoinEvent<K,T> event = waitingEvent.getKey();
    +          joinStream(event.value,event.isStream1Data,future.get());
    +        } catch (InterruptedException | ExecutionException e) {
    +          throw new RuntimeException("end window", e);
    +        }
    +        waitIterator.remove();
    +        if (!finalize) {
    +          break;
    --- End diff --
    
    You can break only when you find a future which is not done... Till then you can keep processing for all the waitingEvents which has done future.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72212094
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    --- End diff --
    
    In seconds? ms?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289923
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    --- End diff --
    
    Yes. Made it into transient.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289958
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297432
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], outputField));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    --- End diff --
    
    description missing for return.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844379
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,363 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isKeyContainsMultiValue)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isKeyContainsMultiValue;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +      return null;
    +    }
    +    if (isKeyContainsMultiValue) {
    +      return (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }
    +    value = new ArrayList<>();
    +    value.add((V)streamCodec.fromByteArray(valueSlice));
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    +  {
    +    return isKeyContainsMultiValue;
    +  }
    +
    +  public void setKeyContainsMultiValue(boolean keyContainsMultiValue)
    +  {
    +    isKeyContainsMultiValue = keyContainsMultiValue;
    +  }
    +
    +  public long getTimeBucket()
    +  {
    +    return timeBucket;
    +  }
    +
    +  public void setTimeBucket(long timeBucket)
    +  {
    +    this.timeBucket = timeBucket;
    +  }
    +
    +  public StreamCodec getStreamCodec()
    +  {
    +    return streamCodec;
    +  }
    +
    +  public void setStreamCodec(StreamCodec streamCodec)
    +  {
    +    this.streamCodec = streamCodec;
    +  }
    +
    +  public class CompositeFuture implements Future<List>
    +  {
    +    public Future<Slice> slice;
    +
    +    public CompositeFuture(Future<Slice> slice)
    +    {
    +      this.slice = slice;
    +    }
    +
    +    @Override
    +    public boolean cancel(boolean b)
    +    {
    +      return slice.cancel(b);
    +    }
    +
    +    @Override
    +    public boolean isCancelled()
    +    {
    +      return slice.isCancelled();
    +    }
    +
    +    @Override
    +    public boolean isDone()
    +    {
    +      return slice.isDone();
    +    }
    +
    +    /**
    +     * Converts the single element into the list.
    +     * @return
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73290027
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +        return null;
    +      }
    +      value = new ArrayList<>();
    +      value.add((V)streamCodec.fromByteArray(valueSlice));
    +    }
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    +  {
    +    return isKeyContainsMultiValue;
    +  }
    +
    +  public void setKeyContainsMultiValue(boolean keyContainsMultiValue)
    +  {
    +    isKeyContainsMultiValue = keyContainsMultiValue;
    +  }
    +
    +  public long getTimeBucket()
    +  {
    +    return timeBucket;
    +  }
    +
    +  public void setTimeBucket(long timeBucket)
    +  {
    +    this.timeBucket = timeBucket;
    +  }
    +
    +  public StreamCodec getStreamCodec()
    +  {
    +    return streamCodec;
    +  }
    +
    +  public void setStreamCodec(StreamCodec streamCodec)
    +  {
    +    this.streamCodec = streamCodec;
    +  }
    +
    +  public class CompositeFuture implements Future<List>
    +  {
    +    public Future<Slice> slice;
    +
    +    public CompositeFuture(Future<Slice> slice)
    +    {
    +      this.slice = slice;
    +    }
    +
    +    @Override
    +    public boolean cancel(boolean b)
    +    {
    +      return slice.cancel(b);
    +    }
    +
    +    @Override
    +    public boolean isCancelled()
    +    {
    +      return slice.isCancelled();
    +    }
    +
    +    @Override
    +    public boolean isDone()
    +    {
    +      return slice.isDone();
    +    }
    +
    +    @Override
    +    public List get() throws InterruptedException, ExecutionException
    +    {
    +      List<V> value = null;
    +      Slice valueSlice = slice.get();
    +      if (isKeyContainsMultiValue) {
    +        value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }  else {
    +        if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    --- 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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72383300
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    --- End diff --
    
    Can you please add a check if timeFields has required number of elements. Might save a NPE.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72402708
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (isKeyContainsMultiValue) {
    +      value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }  else {
    +      if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +        return null;
    +      }
    +      value = new ArrayList<>();
    +      value.add((V)streamCodec.fromByteArray(valueSlice));
    +    }
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    +  {
    +    long timeBucketId = store.getTimeBucketAssigner().getTimeBucketFor(timeBucket);
    +    if (timeBucketId != -1) {
    +      store.putInBucket(bucketId, timeBucketId, keySlice, valueSlice);
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public boolean remove(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(@Nullable K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean putAll(Multimap<? extends K, ? extends V> multimap)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> replaceValues(K k, Iterable<? extends V> iterable)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Map<K, Collection<V>> asMap()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  public int getBucketId(K k)
    +  {
    +    return k.hashCode() % store.getNumBuckets();
    +  }
    +
    +
    +  public boolean isKeyContainsMultiValue()
    +  {
    +    return isKeyContainsMultiValue;
    +  }
    +
    +  public void setKeyContainsMultiValue(boolean keyContainsMultiValue)
    +  {
    +    isKeyContainsMultiValue = keyContainsMultiValue;
    +  }
    +
    +  public long getTimeBucket()
    +  {
    +    return timeBucket;
    +  }
    +
    +  public void setTimeBucket(long timeBucket)
    +  {
    +    this.timeBucket = timeBucket;
    +  }
    +
    +  public StreamCodec getStreamCodec()
    +  {
    +    return streamCodec;
    +  }
    +
    +  public void setStreamCodec(StreamCodec streamCodec)
    +  {
    +    this.streamCodec = streamCodec;
    +  }
    +
    +  public class CompositeFuture implements Future<List>
    +  {
    +    public Future<Slice> slice;
    +
    +    public CompositeFuture(Future<Slice> slice)
    +    {
    +      this.slice = slice;
    +    }
    +
    +    @Override
    +    public boolean cancel(boolean b)
    +    {
    +      return slice.cancel(b);
    +    }
    +
    +    @Override
    +    public boolean isCancelled()
    +    {
    +      return slice.isCancelled();
    +    }
    +
    +    @Override
    +    public boolean isDone()
    +    {
    +      return slice.isDone();
    +    }
    +
    +    @Override
    +    public List get() throws InterruptedException, ExecutionException
    +    {
    +      List<V> value = null;
    +      Slice valueSlice = slice.get();
    +      if (isKeyContainsMultiValue) {
    +        value = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }  else {
    +        if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    --- End diff --
    
    Same comments as the above get() method.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73289926
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    --- End diff --
    
    Yes. It will restore back from the state when it crash and come back.


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72401094
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,357 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isPrimaryKey)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isPrimaryKey;
    --- End diff --
    
    Intuitively, this should be ```! isPrimaryKey``` as only then key can contain multi values.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73297822
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    --- End diff --
    
    CheckpointListener is deprecated, please use CheckpointNotificationListener 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844212
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    + * <b>isStream1KeyPrimary</b>: : Specifies whether the stream1 key is primary or not<br>
    + * <b>isStream2KeyPrimary</b>: : Specifies whether the stream2 key is primary or not<br>
    + *
    + * <b> Example: </b> <br>
    + *  Left input port receives customer details and right input port receives Order details.
    + *  Schema for the Customer be in the form of {ID, Name, CTime}
    + *  Schema for the Order be in the form of {OID, CID, OTime}
    + *  Now, Join the tuples of Customer and Order streams where Customer.ID = Order.CID and the constraint is
    + *  matched tuples must have timestamp within 5 minutes.
    + *  Here, key Fields = ID, CID and Time Fields = CTime, OTime, expiryTime = 5 minutes </b> <br>
    + *
    + *  @displayName Abstract Inner Join Operator
    + *  @tags join
    + */
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +  @AutoMetric
    +  private long tuplesJoinedPerSec;
    +  private double windowTimeSec;
    +  private int tuplesCount;
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +  private Long stream1ExpiryTime;
    +  private Long stream2ExpiryTime;
    +  private boolean isStream1KeyPrimary = true;
    +  private boolean isStream2KeyPrimary = true;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key if found it in opposite store
    +   * 4) Merge the given tuple and values found from step (3)
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    if (!store.put(key, tuple)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    joinStream(tuple,isStream1Data, valuestore.get(key));
    +  }
    +
    +  /**
    +   * Merge the given tuple and list of values.
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @param value list of tuples
    +   */
    +  protected void joinStream(T tuple, boolean isStream1Data, List<T> value)
    +  {
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    --- End diff --
    
    Ok


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844328
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], outputField));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    +   */
    +  @Override
    +  public Object mergeTuples(Object tuple1, Object tuple2)
    +  {
    +    Object o;
    +    try {
    +      o = outputClass.newInstance();
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[0].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple1));
    +      }
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[1].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple2));
    +      }
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return o;
    +  }
    +
    +  /**
    +   * Emit the given tuple through the outputPort
    +   * @param tuple given tuple
    +   */
    +  @Override
    +  public void emitTuple(Object tuple)
    +  {
    +    outputPort.emit(tuple);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    generateSettersAndGetters();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    time += timeIncrement;
    +  }
    +
    +  /**
    +   * Returns the streamcodec for the streams
    +   * @param isStream1data Specifies whether the codec needs for stream1 or stream2.
    +   * @return the object of InnerJoinStreamCodec
    +   */
    +  public StreamCodec<Object> getInnerJoinStreamCodec(boolean isStream1data)
    --- 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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844336
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() != 0) {
    +          Class tc = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), tc);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class ic = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class oc = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], ic),
    +              PojoUtils.createSetter(outputClass, includeFields[i][j], oc));
    +        }
    +      } catch (NoSuchFieldException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Extract the key value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public Object extractKey(Object tuple, boolean isStream1Data)
    +  {
    +    return isStream1Data ? inputFieldObjects[0].keyGet.get(tuple) :
    +      inputFieldObjects[1].keyGet.get(tuple);
    +  }
    +
    +  /**
    +   * Merge the given tuples
    +   * @param tuple1 tuple belongs to stream1
    +   * @param tuple2 tuple belongs to stream1
    +   * @return
    +   */
    +  @Override
    +  public Object mergeTuples(Object tuple1, Object tuple2)
    +  {
    +    Object o;
    +    try {
    +      o = outputClass.newInstance();
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[0].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple1));
    +      }
    +      for (Map.Entry<PojoUtils.Getter,PojoUtils.Setter> g: inputFieldObjects[1].fieldMap.entrySet()) {
    +        g.getValue().set(o, g.getKey().get(tuple2));
    +      }
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return o;
    +  }
    +
    +  /**
    +   * Emit the given tuple through the outputPort
    +   * @param tuple given tuple
    +   */
    +  @Override
    +  public void emitTuple(Object tuple)
    +  {
    +    outputPort.emit(tuple);
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    generateSettersAndGetters();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    time += timeIncrement;
    +  }
    +
    +  /**
    +   * Returns the streamcodec for the streams
    +   * @param isStream1data Specifies whether the codec needs for stream1 or stream2.
    +   * @return the object of InnerJoinStreamCodec
    +   */
    +  public StreamCodec<Object> getInnerJoinStreamCodec(boolean isStream1data)
    +  {
    +    if (isStream1data) {
    +      return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[0]);
    +    }
    +    return new InnerJoinStreamCodec(getKeyFieldsStr().split(",")[1]);
    +  }
    +
    +  private class FieldObjectMap
    +  {
    +    public Class<?> inputClass;
    +    public PojoUtils.Getter keyGet;
    +    public PojoUtils.Getter timeFieldGet;
    +    public Map<PojoUtils.Getter,PojoUtils.Setter> fieldMap;
    --- End diff --
    
    In MergeTuples(), it's difficult to figure out that the getter method belongs to stream1 or stream2.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844259
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,354 @@
    +/**
    + * 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.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateMultiValue;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.utils.serde.Serde;
    +import org.apache.hadoop.fs.Path;
    +import com.google.common.collect.Maps;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of inner join operator over Managed state which extends from
    + * AbstractInnerJoinOperator.
    + *
    + * <b>Properties:</b><br>
    + * <b>noOfBuckets</b>: Number of buckets required for Managed state. <br>
    + * <b>bucketSpanTime</b>: Indicates the length of the time bucket. <br>
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient Map<JoinEvent<K,T>, Future<List>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  /**
    +   * Create Managed states and stores for both the streams.
    +   */
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +    stream1Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +    stream2Store.getTimeBucketAssigner().setExpireBefore(Duration.millis(getExpiryTime()));
    +
    +    component = new ManagedSpillableComplexComponent();
    +    stream1Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream1Store, !isStream1KeyPrimary());
    +    stream2Data = ((ManagedSpillableComplexComponent)component).newSpillableByteArrayListMultimap(stream2Store, !isStream2KeyPrimary());
    +  }
    +
    +  /**
    +   * Process the tuple which are received from input ports with the following steps:
    +   * 1) Extract key from the given tuple
    +   * 2) Insert <key,tuple> into the store where store is the stream1Data if the tuple
    +   * receives from stream1 or viceversa.
    +   * 3) Get the values of the key in asynchronous if found it in opposite store
    +   * 4) If the future is done then Merge the given tuple and values found from step (3) otherwise
    +   *    put it in waitingEvents
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   */
    +  @Override
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    long timeBucket = extractTime(tuple,isStream1Data);
    +    if (!((ManagedTimeStateMultiValue)store).put(key, tuple,timeBucket)) {
    +      return;
    +    }
    +    Spillable.SpillableByteArrayListMultimap<K, T> valuestore = isStream1Data ? stream2Data : stream1Data;
    +    Future<List> future = ((ManagedTimeStateMultiValue)valuestore).getAsync(key);
    +    if (future.isDone()) {
    +      try {
    +        joinStream(tuple,isStream1Data, future.get());
    +      } catch (InterruptedException e) {
    +        throw new RuntimeException(e);
    +      } catch (ExecutionException e) {
    +        throw new RuntimeException(e);
    +      }
    +    } else {
    +      waitingEvents.put(new JoinEvent<>(key,tuple,isStream1Data),future);
    +    }
    +  }
    +
    +  @Override
    +  public void handleIdleTime()
    +  {
    +    if (waitingEvents.size() > 0) {
    +      processWaitEvents(false);
    +    }
    +  }
    +
    +  @Override
    +  public void beforeCheckpoint(long l)
    +  {
    +    stream1Store.beforeCheckpoint(l);
    +    stream2Store.beforeCheckpoint(l);
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +    stream1Store.checkpointed(l);
    +    stream2Store.checkpointed(l);
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    stream1Store.committed(l);
    +    stream2Store.committed(l);
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +    ((FileAccessFSImpl)stream1Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream1State);
    +    ((FileAccessFSImpl)stream2Store.getFileAccess()).setBasePath(context.getValue(DAG.APPLICATION_PATH) + Path.SEPARATOR + stateDir + Path.SEPARATOR + String.valueOf(context.getId()) + Path.SEPARATOR + stream2State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream1State);
    +    stream1Store.getCheckpointManager().setRecoveryPath("managed_state_" + stream2State);
    +    stream1Store.setup(context);
    +    stream2Store.setup(context);
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    stream1Store.beginWindow(windowId);
    +    stream2Store.beginWindow(windowId);
    +    super.beginWindow(windowId);
    +  }
    +
    +  /**
    +   * Process the waiting events
    +   * @param finalize finalize Whether or not to wait for future to return
    +   */
    +  private void processWaitEvents(boolean finalize)
    +  {
    +    Iterator<Map.Entry<JoinEvent<K,T>, Future<List>>> waitIterator = waitingEvents.entrySet().iterator();
    +    while (waitIterator.hasNext()) {
    +      Map.Entry<JoinEvent<K,T>, Future<List>> waitingEvent = waitIterator.next();
    +      Future<List> future = waitingEvent.getValue();
    +      if (future.isDone() || finalize) {
    +        try {
    +          JoinEvent<K,T> event = waitingEvent.getKey();
    +          joinStream(event.value,event.isStream1Data,future.get());
    +        } catch (InterruptedException | ExecutionException e) {
    +          throw new RuntimeException("end window", e);
    +        }
    +        waitIterator.remove();
    +        if (!finalize) {
    +          break;
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    processWaitEvents(true);
    +    stream1Store.endWindow();
    +    stream2Store.endWindow();
    +    super.endWindow();
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    stream1Store.teardown();
    +    stream2Store.teardown();
    +    super.teardown();
    +  }
    +
    +  /**
    +   * Return the number of buckets
    +   * @return the noOfBuckets
    +   */
    +  public int getNoOfBuckets()
    +  {
    +    return noOfBuckets;
    +  }
    +
    +  /**
    +   * Set the number of buckets required for managed state
    +   * @param noOfBuckets noOfBuckets
    +   */
    +  public void setNoOfBuckets(int noOfBuckets)
    +  {
    +    this.noOfBuckets = noOfBuckets;
    +  }
    +
    +  /**
    +   * Return the bucketSpanTime
    +   * @return the bucketSpanTime
    +   */
    +  public Long getBucketSpanTime()
    +  {
    +    return bucketSpanTime;
    +  }
    +
    +  /**
    +   * Sets the length of the time bucket required for managed state.
    +   * @param bucketSpanTime given bucketSpanTime
    +   */
    +  public void setBucketSpanTime(Long bucketSpanTime)
    +  {
    +    this.bucketSpanTime = bucketSpanTime;
    +  }
    +
    +  public static class JoinEvent<K,T>
    +  {
    +    public K key;
    +    public T value;
    +    public boolean isStream1Data;
    +
    +    public JoinEvent(K key, T value, boolean isStream1Data)
    +    {
    +      this.key = key;
    +      this.value = value;
    +      this.isStream1Data = isStream1Data;
    +    }
    +  }
    +
    +  public class ManagedSpillableComplexComponent implements SpillableComplexComponent
    --- End diff --
    
    Yes. Initialized without creating component.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73281761
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    --- End diff --
    
    No. In declaration, it creates the array with Map type.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r73844360
  
    --- Diff: library/src/main/java/org/apache/apex/malhar/lib/state/managed/ManagedTimeStateMultiValue.java ---
    @@ -0,0 +1,363 @@
    +/**
    + * 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.state.managed;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Multiset;
    +
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Concrete implementation of SpillableByteArrayListMultimap which is needed for join operator.
    + *
    + * <b>Properties:</b><br>
    + * <b>isKeyContainsMultiValue</b>: Specifies whether the key has multiple value or not. <br>
    + * <b>timeBucket</b>: Specifies the lenght of the time bucket.
    + *
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class ManagedTimeStateMultiValue<K,V> implements Spillable.SpillableByteArrayListMultimap<K,V>
    +{
    +  private transient StreamCodec streamCodec = null;
    +  private boolean isKeyContainsMultiValue = false;
    +  private long timeBucket;
    +  @NotNull
    +  private ManagedTimeStateImpl store;
    +
    +  public ManagedTimeStateMultiValue()
    +  {
    +    if (streamCodec == null) {
    +      streamCodec = new KryoSerializableStreamCodec();
    +    }
    +  }
    +
    +  public ManagedTimeStateMultiValue(@NotNull ManagedTimeStateImpl store, boolean isKeyContainsMultiValue)
    +  {
    +    this();
    +    this.store = Preconditions.checkNotNull(store);
    +    this.isKeyContainsMultiValue = isKeyContainsMultiValue;
    +  }
    +
    +  /**
    +   * Return the list of values from the store
    +   * @param k given key
    +   * @return list of values
    +   */
    +  @Override
    +  public List<V> get(@Nullable K k)
    +  {
    +    List<V> value = null;
    +    Slice valueSlice = store.getSync(getBucketId(k), streamCodec.toByteArray(k));
    +    if (valueSlice == null || valueSlice.length == 0 || valueSlice.buffer == null) {
    +      return null;
    +    }
    +    if (isKeyContainsMultiValue) {
    +      return (List<V>)streamCodec.fromByteArray(valueSlice);
    +    }
    +    value = new ArrayList<>();
    +    value.add((V)streamCodec.fromByteArray(valueSlice));
    +    return  value;
    +  }
    +
    +  /**
    +   * Returns the Future form the store.
    +   * @param k given key
    +   * @return
    +   */
    +  public CompositeFuture getAsync(@Nullable K k)
    +  {
    +    return new CompositeFuture(store.getAsync(getBucketId(k), streamCodec.toByteArray(k)));
    +  }
    +
    +  @Override
    +  public Set<K> keySet()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Multiset<K> keys()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<V> values()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Collection<Map.Entry<K, V>> entries()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public List<V> removeAll(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public void clear()
    +  {
    +
    +  }
    +
    +  @Override
    +  public int size()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean isEmpty()
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsKey(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsValue(@Nullable Object o)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public boolean containsEntry(@Nullable Object o, @Nullable Object o1)
    +  {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store.
    +   * @param k key
    +   * @param v value
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  @Override
    +  public boolean put(@Nullable K k, @Nullable V v)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Inserts the (k,v) into the store using the specified timebucket.
    +   * @param k key
    +   * @param v value
    +   * @param timeBucket timebucket
    +   * @return true if the given (k,v) is successfully inserted into the store otherwise false.
    +   */
    +  public boolean put(@Nullable K k, @Nullable V v, long timeBucket)
    +  {
    +    if (isKeyContainsMultiValue) {
    +      Slice keySlice = streamCodec.toByteArray(k);
    +      int bucketId = getBucketId(k);
    +      Slice valueSlice = store.getSync(bucketId, keySlice);
    +      List<V> listOb;
    +      if (valueSlice == null || valueSlice.length == 0) {
    +        listOb = new ArrayList<>();
    +      } else {
    +        listOb = (List<V>)streamCodec.fromByteArray(valueSlice);
    +      }
    +      listOb.add(v);
    +      return insertInStore(bucketId, timeBucket, keySlice, streamCodec.toByteArray(listOb));
    +    }
    +    return insertInStore(getBucketId(k), timeBucket, streamCodec.toByteArray(k),streamCodec.toByteArray(v));
    +  }
    +
    +  /**
    +   * Insert (keySlice,valueSlice) into the store using bucketId and timeBucket.
    +   * @param bucketId bucket Id
    +   * @param timeBucket time bucket
    +   * @param keySlice key slice
    +   * @param valueSlice value slice
    +   * @return true if the given (keySlice,valueSlice) is successfully inserted into the
    +   *         store otherwise false.
    +   */
    +  public boolean insertInStore(long bucketId, long timeBucket, Slice keySlice, Slice valueSlice)
    --- End diff --
    
    Yes. I made this to private.


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73309435
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/InnerJoinStreamCodec.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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 com.datatorrent.lib.codec.KryoSerializableStreamCodec;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Stream codec based on keyExpression for POJO Inner Join Operator.
    + *
    + * @Since 3.5.0
    --- End diff --
    
    Please remove @since tag.. This will be added during release 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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70043086
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,187 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +public abstract class AbstractInnerJoinOperator<K,T> extends BaseOperator
    +{
    +  protected transient String[][] includeFields;
    +  protected transient List<String> keyFields;
    +  protected transient List<String> timeFields;
    +
    +  private Long expiryTime;
    +  protected SpillableComplexComponent component;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream1Data;
    +  protected Spillable.SpillableByteArrayListMultimap<K,T> stream2Data;
    +  private boolean isStream1KeyPrimary = false;
    +  private boolean isStream2KeyPrimary = false;
    +
    +  @NotNull
    +  private String keyFieldsStr;
    +  @NotNull
    +  private String includeFieldStr;
    +  private String timeFieldsStr;
    +
    +  protected void processTuple(T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K,T> store = isStream1Data ? stream1Data : stream2Data;
    +    K key = extractKey(tuple,isStream1Data);
    +    store.put(key, tuple);
    +    joinStream(key,tuple,isStream1Data);
    +  }
    +
    +  protected void joinStream(K key, T tuple, boolean isStream1Data)
    +  {
    +    Spillable.SpillableByteArrayListMultimap<K, T> store = isStream1Data ? stream2Data : stream1Data;
    +    List<T> value = store.get(key);
    +
    +    // Join the input tuple with the joined tuples
    +    if (value != null) {
    +      for (T joinedValue : value) {
    +        T result = isStream1Data ? joinTuples(Arrays.asList(tuple, joinedValue)) :
    --- End diff --
    
    why list as argument to joinTuples?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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

    https://github.com/apache/apex-malhar/pull/330#discussion_r70042432
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractManagedStateInnerJoinOperator.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.apex.malhar.lib.state.managed.ManagedTimeStateImpl;
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.google.common.collect.Maps;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.lib.fileaccess.FileAccessFSImpl;
    +import com.datatorrent.lib.join.managed.ManagedSpillableComplexComponent;
    +import com.datatorrent.lib.join.managed.ManagedTimeStateMultiMap;
    +import com.datatorrent.netlet.util.Slice;
    +
    +public abstract class AbstractManagedStateInnerJoinOperator<K,T> extends AbstractInnerJoinOperator<K,T> implements
    +    Operator.CheckpointNotificationListener, Operator.CheckpointListener,Operator.IdleTimeHandler
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(AbstractManagedStateInnerJoinOperator.class);
    +  public static final String stateDir = "managedState";
    +  public static final String stream1State = "stream1Data";
    +  public static final String stream2State = "stream2Data";
    +  private transient long sleepMillis;
    +  private transient Map<JoinEvent<K,T>, Future<Slice>> waitingEvents = Maps.newLinkedHashMap();
    +  private int noOfBuckets = 1;
    +  private Long bucketSpanTime;
    +  protected ManagedTimeStateImpl stream1Store;
    +  protected ManagedTimeStateImpl stream2Store;
    +
    +  @Override
    +  public void createStores()
    +  {
    +    stream1Store = new ManagedTimeStateImpl();
    +    stream2Store = new ManagedTimeStateImpl();
    +    stream1Store.setNumBuckets(noOfBuckets);
    +    stream2Store.setNumBuckets(noOfBuckets);
    +    if (bucketSpanTime != null) {
    +      stream1Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +      stream2Store.getTimeBucketAssigner().setBucketSpan(Duration.millis(bucketSpanTime));
    +    }
    +
    +    if (getExpiryTime() != null) {
    --- End diff --
    
    can it be different for each stores?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72211984
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/AbstractInnerJoinOperator.java ---
    @@ -0,0 +1,331 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.List;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.apache.apex.malhar.lib.state.spillable.Spillable;
    +import org.apache.apex.malhar.lib.state.spillable.SpillableComplexComponent;
    +import org.apache.apex.malhar.lib.state.spillable.inmem.InMemSpillableComplexComponent;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * <p>
    + * An abstract implementation of inner join operator. Operator receives tuples from two streams,
    + * applies the join operation based on constraint and emit the joined value.
    + * Concrete classes should provide implementation to extractKey, extractTime, mergeTuples methods.
    + *
    + * <b>Properties:</b><br>
    + * <b>includeFieldStr</b>: List of comma separated fields to be added to the output tuple.
    + *                         Ex: Field1,Field2;Field3,Field4<br>
    + * <b>keyFields</b>: List of comma separated key field for both the streams. Ex: Field1,Field2<br>
    + * <b>timeFields</b>: List of comma separated time field for both the streams. Ex: Field1,Field2<br>
    + * <b>expiryTime</b>: Expiry time for stored tuples<br>
    --- End diff --
    
    In seconds? ms?


---
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 #330: Initial cut of Inner Join operator for REVIEW...

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/330#discussion_r72087034
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    --- End diff --
    
    keyFields is populated in setup method of the operator but this method is called from activate? activate of operator is called before setup.
    Can you please check if this is giving desired result? Am I missing something?


---
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 #330: APEXMALHAR-2100 Implementation of Inner Join ...

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/330#discussion_r73828318
  
    --- Diff: library/src/main/java/com/datatorrent/lib/join/POJOInnerJoinOperator.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.HashMap;
    +import java.util.Map;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.ClassUtils;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.StreamCodec;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * Concrete implementation of AbstractManagedStateInnerJoinOperator and receives objects from both streams.
    + *
    + * @displayName POJO Inner Join Operator
    + * @tags join
    + */
    +@org.apache.hadoop.classification.InterfaceStability.Evolving
    +public class POJOInnerJoinOperator extends AbstractManagedStateInnerJoinOperator<Object,Object> implements Operator.ActivationListener<Context>
    +{
    +  private static final transient Logger LOG = LoggerFactory.getLogger(POJOInnerJoinOperator.class);
    +  private transient long timeIncrement;
    +  private transient FieldObjectMap[] inputFieldObjects = (FieldObjectMap[])Array.newInstance(FieldObjectMap.class, 2);
    +  protected transient Class<?> outputClass;
    +  private long time = System.currentTimeMillis();
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> outputPort = new DefaultOutputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      outputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[0].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,true);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(true);
    +    }
    +  };
    +
    +  @InputPortFieldAnnotation(schemaRequired = true)
    +  public transient DefaultInputPort<Object> input2 = new DefaultInputPort<Object>()
    +  {
    +    @Override
    +    public void setup(Context.PortContext context)
    +    {
    +      inputFieldObjects[1].inputClass = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple,false);
    +    }
    +
    +    @Override
    +    public StreamCodec<Object> getStreamCodec()
    +    {
    +      return getInnerJoinStreamCodec(false);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    timeIncrement = context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +      context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS);
    +    super.setup(context);
    +    for (int i = 0; i < 2; i++) {
    +      inputFieldObjects[i] = new FieldObjectMap();
    +    }
    +  }
    +
    +  /**
    +   * Extract the time value from the given tuple
    +   * @param tuple given tuple
    +   * @param isStream1Data Specifies whether the given tuple belongs to stream1 or not.
    +   * @return
    +   */
    +  @Override
    +  public long extractTime(Object tuple, boolean isStream1Data)
    +  {
    +    return timeFields == null ? time : (long)(isStream1Data ? inputFieldObjects[0].timeFieldGet.get(tuple) :
    +          inputFieldObjects[1].timeFieldGet.get(tuple));
    +  }
    +
    +  /**
    +   * Create getters for the key and time fields and setters for the include fields.
    +   */
    +  protected void generateSettersAndGetters()
    +  {
    +    for (int i = 0; i < 2; i++) {
    +      Class inputClass = inputFieldObjects[i].inputClass;
    +      try {
    +        Class c = ClassUtils.primitiveToWrapper(inputClass.getField(keyFields.get(i)).getType());
    +        inputFieldObjects[i].keyGet = PojoUtils.createGetter(inputClass, keyFields.get(i), c);
    +        if (timeFields != null && timeFields.size() == 2) {
    +          Class timeField = ClassUtils.primitiveToWrapper(inputClass.getField(timeFields.get(i)).getType());
    +          inputFieldObjects[i].timeFieldGet = PojoUtils.createGetter(inputClass, timeFields.get(i), timeField);
    +        }
    +        for (int j = 0; j < includeFields[i].length; j++) {
    +          Class inputField = ClassUtils.primitiveToWrapper(inputClass.getField(includeFields[i][j]).getType());
    +          Class outputField = ClassUtils.primitiveToWrapper(outputClass.getField(includeFields[i][j]).getType());
    +          inputFieldObjects[i].fieldMap.put(PojoUtils.createGetter(inputClass, includeFields[i][j], inputField),
    --- End diff --
    
    Can you check whether types are also same for inputField and outputField, otherwise, they're not matching 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.
---