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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to