yifan-c commented on code in PR #17: URL: https://github.com/apache/cassandra-analytics/pull/17#discussion_r1409303414
########## cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java: ########## @@ -136,35 +185,122 @@ 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>> newRanges, + ReplicaAwareFailureHandler<RingInstance> failureHandler) throws IOException + { + 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)); + + // token range for this partition is not among the write-replica-set ranges + if (!newRanges.contains(tokenRange)) + { + Set<Range<BigInteger>> subRanges = getIntersectingSubRanges(newRanges, tokenRange); + // We have split ranges - likely resulting from pending nodes + if (subRanges.size() > 1) + { + // Create session using sub-range that contains the token from current row + Range<BigInteger> matchingRange = subRanges.stream().filter(r -> r.contains(token)).findFirst().get(); + Preconditions.checkState(matchingRange != null, + String.format("Received Token %s outside of expected range %s", token, matchingRange)); + + if (streamSession != null && streamSession.getTokenRange() == matchingRange) + { + return streamSession; + } + else + { + LOGGER.debug(String.format("[{}] Creating stream session for range: %s", matchingRange), taskContext.partitionId()); + if (streamSession != null && batchSize != 0) + { + finalizeSSTable(streamSession, taskContext.partitionId(), sstableWriter, batchNumber, batchSize); + sstableWriter = null; + batchSize = 0; + } + return new StreamSession(writerContext, getStreamId(taskContext), matchingRange, failureHandler); + } + } + } + + return (streamSession != null) ? streamSession : createStreamSession(taskContext); + } + + /** + * 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); + + return !(startMapping.keySet().equals(endMapping.keySet()) && + instancesFromMapping(startMapping).equals(instancesFromMapping(endMapping))); Review Comment: nit: I think this is easier to read with less parenthesis. ```suggestion return !Objects.equals(startMapping.keySet(), endMapping.keySet()) || !Objects.equals(instancesFromMapping(startMapping), instancesFromMapping(endMapping)); ``` A follow up question. In the scenario that a node is joining at the beginning of a task and finishes joining at the end, the `haveTokenRangeMappingsChanged` would return false. Am I right? ########## cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java: ########## @@ -136,35 +185,122 @@ 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>> newRanges, + ReplicaAwareFailureHandler<RingInstance> failureHandler) throws IOException + { + 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)); + + // token range for this partition is not among the write-replica-set ranges + if (!newRanges.contains(tokenRange)) Review Comment: This check is performed for _every_ row. Is it really needed here? The `newRanges` and `tokenRange` remain the same while writing (the `while` loop) ########## cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java: ########## @@ -110,20 +132,47 @@ public StreamResult write(Iterator<Tuple2<DecoratedKey, Object[]>> sourceIterato Map<String, Object> valueMap = new HashMap<>(); try { + List<RingInstance> exclusions = failureHandler.getFailedInstances(); + Set<Range<BigInteger>> newRanges = initialTokenRangeMapping.getRangeMap().asMapOfRanges().entrySet() + .stream() + .filter(e -> !exclusions.contains(e.getValue())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + while (dataIterator.hasNext()) { + Tuple2<DecoratedKey, Object[]> rowData = dataIterator.next(); + streamSession = maybeCreateStreamSession(taskContext, streamSession, rowData, newRanges, failureHandler); + + sessions.add(streamSession); maybeCreateTableWriter(partitionId, baseDir); - writeRow(valueMap, dataIterator, partitionId, range); + writeRow(rowData, valueMap, partitionId, streamSession.getTokenRange()); checkBatchSize(streamSession, partitionId, job); } - if (sstableWriter != null) + // Finalize SSTable for the last StreamSession + if (sstableWriter != null || (streamSession != null && batchSize != 0)) Review Comment: `streamSession` cannot be null. The check is redundant. When `sstableWriter` is not null, the `batchSize` cannot be 0. The condition `batchSize != 0` is redundant too. ########## cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java: ########## @@ -110,20 +132,47 @@ public StreamResult write(Iterator<Tuple2<DecoratedKey, Object[]>> sourceIterato Map<String, Object> valueMap = new HashMap<>(); try { + List<RingInstance> exclusions = failureHandler.getFailedInstances(); + Set<Range<BigInteger>> newRanges = initialTokenRangeMapping.getRangeMap().asMapOfRanges().entrySet() + .stream() + .filter(e -> !exclusions.contains(e.getValue())) Review Comment: `exclusions` is a list, and `e.getValue()` returns a list too. `list.contains(list)` is always evaluated to be `false`. ########## cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java: ########## @@ -136,35 +185,122 @@ 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>> newRanges, + ReplicaAwareFailureHandler<RingInstance> failureHandler) throws IOException + { + 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)); + + // token range for this partition is not among the write-replica-set ranges + if (!newRanges.contains(tokenRange)) + { + Set<Range<BigInteger>> subRanges = getIntersectingSubRanges(newRanges, tokenRange); + // We have split ranges - likely resulting from pending nodes + if (subRanges.size() > 1) + { + // Create session using sub-range that contains the token from current row + Range<BigInteger> matchingRange = subRanges.stream().filter(r -> r.contains(token)).findFirst().get(); + Preconditions.checkState(matchingRange != null, + String.format("Received Token %s outside of expected range %s", token, matchingRange)); + + if (streamSession != null && streamSession.getTokenRange() == matchingRange) + { + return streamSession; + } + else + { + LOGGER.debug(String.format("[{}] Creating stream session for range: %s", matchingRange), taskContext.partitionId()); Review Comment: `String.format` is redundant ########## cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java: ########## @@ -110,20 +132,47 @@ public StreamResult write(Iterator<Tuple2<DecoratedKey, Object[]>> sourceIterato Map<String, Object> valueMap = new HashMap<>(); try { + List<RingInstance> exclusions = failureHandler.getFailedInstances(); + Set<Range<BigInteger>> newRanges = initialTokenRangeMapping.getRangeMap().asMapOfRanges().entrySet() + .stream() + .filter(e -> !exclusions.contains(e.getValue())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + while (dataIterator.hasNext()) { + Tuple2<DecoratedKey, Object[]> rowData = dataIterator.next(); + streamSession = maybeCreateStreamSession(taskContext, streamSession, rowData, newRanges, failureHandler); + + sessions.add(streamSession); Review Comment: There is one active session at a time. Why collecting the sessions rather than close the previous session immediately? -- 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