arjunashok commented on code in PR #17:
URL: 
https://github.com/apache/cassandra-analytics/pull/17#discussion_r1420858512


##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java:
##########
@@ -136,35 +181,139 @@ public StreamResult write(Iterator<Tuple2<DecoratedKey, 
Object[]>> sourceIterato
         }
     }
 
+    private Map<Range<BigInteger>, List<RingInstance>> 
taskTokenRangeMapping(TokenRangeMapping<RingInstance> tokenRange,
+                                                                             
Range<BigInteger> taskTokenRange)
+    {
+        return tokenRange.getSubRanges(taskTokenRange).asMapOfRanges();
+    }
+
+    private Set<RingInstance> instancesFromMapping(Map<Range<BigInteger>, 
List<RingInstance>> mapping)
+    {
+        return mapping.values()
+                      .stream()
+                      .flatMap(Collection::stream)
+                      .collect(Collectors.toSet());
+    }
+
+    /**
+     * Creates a new session if we have the current token range intersecting 
the ranges from write replica-set.
+     * If we do find the need to split a range into sub-ranges, we create the 
corresponding session for the sub-range
+     * if the token from the row data belongs to the range.
+     */
+    private StreamSession maybeCreateStreamSession(TaskContext taskContext,
+                                                   StreamSession streamSession,
+                                                   Tuple2<DecoratedKey, 
Object[]> rowData,
+                                                   Set<Range<BigInteger>> 
subRanges,
+                                                   
ReplicaAwareFailureHandler<RingInstance> failureHandler,
+                                                   List<StreamResult> results)
+    throws IOException, ExecutionException, InterruptedException
+    {
+        BigInteger token = rowData._1().getToken();
+        Range<BigInteger> tokenRange = getTokenRange(taskContext);
+
+        Preconditions.checkState(tokenRange.contains(token),
+                                 String.format("Received Token %s outside of 
expected range %s", token, tokenRange));
+
+        // We have split ranges likely resulting from pending nodes
+        // Evaluate creating a new session if the token from current row is 
part of a sub-range
+        if (subRanges.size() > 1)
+        {
+            // Create session using sub-range that contains the token from 
current row
+            Range<BigInteger> matchingSubRange = subRanges.stream().filter(r 
-> r.contains(token)).findFirst().get();
+            Preconditions.checkState(matchingSubRange != null,
+                                     String.format("Received Token %s outside 
of expected range %s", token, matchingSubRange));
+            streamSession = maybeCreateSubRangeSession(taskContext, 
streamSession, failureHandler, results, matchingSubRange);
+        }
+
+        // If we do not have any stream session at this point, we create a 
session using the partition's token range
+        return (streamSession == null) ? createStreamSession(taskContext) : 
streamSession;
+    }
+
+    /**
+     * Given that the token belongs to a sub-range, creates a new stream 
session if either
+     * 1) we do not have an existing stream session, or 2) the existing stream 
session corresponds to a range that
+     * does NOT match the sub-range the token belongs to.
+     */
+    private StreamSession maybeCreateSubRangeSession(TaskContext taskContext,
+                                                     StreamSession 
streamSession,
+                                                     
ReplicaAwareFailureHandler<RingInstance> failureHandler,
+                                                     List<StreamResult> 
results,
+                                                     Range<BigInteger> 
matchingSubRange)
+    throws IOException, ExecutionException, InterruptedException
+    {
+        if (streamSession == null || streamSession.getTokenRange() != 
matchingSubRange)
+        {
+            LOGGER.debug("[{}] Creating stream session for range: {}", 
taskContext.partitionId(), matchingSubRange);
+            // Schedule data to be sent if we are processing a batch that has 
not been scheduled yet.
+            if (streamSession != null)
+            {
+                // Complete existing batched writes (if any) before the 
existing stream session is closed
+                if (batchSize != 0)
+                {
+                    finalizeSSTable(streamSession, taskContext.partitionId(), 
sstableWriter, batchNumber, batchSize);
+                    sstableWriter = null;
+                    batchSize = 0;
+                }
+                results.add(streamSession.close());
+            }
+            streamSession = new StreamSession(writerContext, 
getStreamId(taskContext), matchingSubRange, failureHandler);
+        }
+        return streamSession;
+    }
+
+    /**
+     * Get ranges from the set that intersect and/or overlap with the provided 
token range
+     */
+    private Set<Range<BigInteger>> 
getIntersectingSubRanges(Set<Range<BigInteger>> ranges, Range<BigInteger> 
tokenRange)
+    {
+        return ranges.stream()
+                     .filter(r -> r.isConnected(tokenRange) && 
!r.intersection(tokenRange).isEmpty())
+                     .collect(Collectors.toSet());
+    }
+
+    private boolean 
haveTokenRangeMappingsChanged(TokenRangeMapping<RingInstance> startTaskMapping, 
TaskContext taskContext)
+    {
+        Range<BigInteger> taskTokenRange = getTokenRange(taskContext);
+        // Get the uncached, current view of the ring to compare with initial 
ring
+        TokenRangeMapping<RingInstance> endTaskMapping = 
writerContext.cluster().getTokenRangeMapping(false);
+        Map<Range<BigInteger>, List<RingInstance>> startMapping = 
taskTokenRangeMapping(startTaskMapping, taskTokenRange);
+        Map<Range<BigInteger>, List<RingInstance>> endMapping = 
taskTokenRangeMapping(endTaskMapping, taskTokenRange);
+
+        // Token ranges are identical and overall instance list is same
+        return !(startMapping.keySet().equals(endMapping.keySet()) &&
+               
instancesFromMapping(startMapping).equals(instancesFromMapping(endMapping)));
+    }
+
     private void validateAcceptableTimeSkewOrThrow(List<RingInstance> replicas)
     {
-        TimeSkewResponse timeSkewResponse = 
writerContext.cluster().getTimeSkew(replicas);
-        Instant localNow = Instant.now();
-        Instant remoteNow = Instant.ofEpochMilli(timeSkewResponse.currentTime);
-        Duration range = 
Duration.ofMinutes(timeSkewResponse.allowableSkewInMinutes);
-        if (localNow.isBefore(remoteNow.minus(range)) || 
localNow.isAfter(remoteNow.plus(range)))
+        if (!replicas.isEmpty())
         {
-            String message = String.format("Time skew between Spark and 
Cassandra is too large. "
-                                           + "Allowable skew is %d minutes. "
-                                           + "Spark executor time is %s, 
Cassandra instance time is %s",
-                                           
timeSkewResponse.allowableSkewInMinutes, localNow, remoteNow);
-            throw new UnsupportedOperationException(message);
+            TimeSkewResponse timeSkewResponse = 
writerContext.cluster().getTimeSkew(replicas);
+            Instant localNow = Instant.now();
+            Instant remoteNow = 
Instant.ofEpochMilli(timeSkewResponse.currentTime);
+            Duration range = 
Duration.ofMinutes(timeSkewResponse.allowableSkewInMinutes);
+            if (localNow.isBefore(remoteNow.minus(range)) || 
localNow.isAfter(remoteNow.plus(range)))
+            {
+                final String message = String.format("Time skew between Spark 
and Cassandra is too large. "
+                                                     + "Allowable skew is %d 
minutes. Spark executor time is %s, Cassandra instance time is %s",
+                                                     
timeSkewResponse.allowableSkewInMinutes, localNow, remoteNow);
+                throw new UnsupportedOperationException(message);
+            }

Review Comment:
   addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to