[
https://issues.apache.org/jira/browse/APEXMALHAR-2100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15405754#comment-15405754
]
ASF GitHub Bot commented on APEXMALHAR-2100:
--------------------------------------------
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>
+ */
[email protected]
+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.
> Development of Inner Join Operator using Spillable Datastructures
> -----------------------------------------------------------------
>
> Key: APEXMALHAR-2100
> URL: https://issues.apache.org/jira/browse/APEXMALHAR-2100
> Project: Apache Apex Malhar
> Issue Type: Task
> Reporter: Chaitanya
> Assignee: Chaitanya
>
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)