-
Notifications
You must be signed in to change notification settings - Fork 513
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Support Tap for SMB writes (addresses #5080) #5144
Merged
Merged
Changes from 7 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
c5d4408
Support Tap for SMB writes (addresses #5080)
clairemcginty c87d41a
update comment
clairemcginty 56fceb2
Refactor; support pre-keyed SCollections
clairemcginty 95e0005
Implement for SMB transforms
clairemcginty 0817556
refactor test
clairemcginty 3754d09
Scala 2.12 compat FlatMap signature
clairemcginty d12c49b
Add test for Tap::flatMap
clairemcginty 559d716
Use self.coder
clairemcginty c3e7ca5
Use BCoder in saveAsPreKeyedSortedBucket
clairemcginty File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,7 +18,7 @@ package com.spotify.scio.smb | |
|
||
import com.spotify.scio.ScioContext | ||
import com.spotify.scio.coders.Coder | ||
import com.spotify.scio.io.{ClosedTap, EmptyTap} | ||
import com.spotify.scio.io.{ClosedTap, EmptyTap, TapOf} | ||
import com.spotify.scio.testing.TestDataManager | ||
import com.spotify.scio.values.{SCollection, SideInput, SideInputContext} | ||
import org.apache.beam.sdk.extensions.smb.{SortedBucketIOUtil, SortedBucketTransform} | ||
|
@@ -82,10 +82,10 @@ object SortMergeTransform { | |
*/ | ||
def via( | ||
transformFn: (KeyType, R, SortedBucketTransform.SerializableConsumer[W]) => Unit | ||
): ClosedTap[Nothing] | ||
): ClosedTap[W] | ||
} | ||
|
||
private[smb] class WriteBuilderImpl[KeyType, R, W]( | ||
private[smb] class WriteBuilderImpl[KeyType, R, W: Coder]( | ||
@transient private val sc: ScioContext, | ||
transform: AbsCoGbkTransform[KeyType, W], | ||
fromResult: CoGbkResult => R | ||
|
@@ -97,7 +97,7 @@ object SortMergeTransform { | |
|
||
override def via( | ||
transformFn: (KeyType, R, SortedBucketTransform.SerializableConsumer[W]) => Unit | ||
): ClosedTap[Nothing] = { | ||
): ClosedTap[W] = { | ||
val fn = new SortedBucketTransform.TransformFn[KeyType, W]() { | ||
override def writeTransform( | ||
keyGroup: KV[KeyType, CoGbkResult], | ||
|
@@ -112,8 +112,9 @@ object SortMergeTransform { | |
|
||
val t = transform.via(fn) | ||
val tfName = sc.tfName(Some("sortMergeTransform")) | ||
sc.applyInternal(tfName, t) | ||
ClosedTap[Nothing](EmptyTap) | ||
val writeResult = sc.applyInternal(tfName, t) | ||
|
||
ClosedTap(SmbIO.tap(t.getFileOperations, writeResult).apply(sc)) | ||
} | ||
} | ||
|
||
|
@@ -130,11 +131,10 @@ object SortMergeTransform { | |
|
||
override def via( | ||
transformFn: (KeyType, R, SortedBucketTransform.SerializableConsumer[W]) => Unit | ||
): ClosedTap[Nothing] = { | ||
): ClosedTap[W] = { | ||
val data = read.parDo(new ViaTransform(transformFn)) | ||
val testOutput = TestDataManager.getOutput(sc.testId.get) | ||
testOutput(SortedBucketIOUtil.testId(output))(data) | ||
ClosedTap[Nothing](EmptyTap) | ||
TestDataManager.getOutput(sc.testId.get)(SortedBucketIOUtil.testId(output)) | ||
ClosedTap(TapOf[W].saveForTest(data)) | ||
} | ||
} | ||
|
||
|
@@ -159,10 +159,10 @@ object SortMergeTransform { | |
SideInputContext[_], | ||
SortedBucketTransform.SerializableConsumer[W] | ||
) => Unit | ||
): ClosedTap[Nothing] | ||
): ClosedTap[W] | ||
} | ||
|
||
private[smb] class WithSideInputsWriteBuilderImpl[KeyType, R, W]( | ||
private[smb] class WithSideInputsWriteBuilderImpl[KeyType, R, W: Coder]( | ||
@transient private val sc: ScioContext, | ||
transform: AbsCoGbkTransform[KeyType, W], | ||
toR: CoGbkResult => R, | ||
|
@@ -175,7 +175,7 @@ object SortMergeTransform { | |
SideInputContext[_], | ||
SortedBucketTransform.SerializableConsumer[W] | ||
) => Unit | ||
): ClosedTap[Nothing] = { | ||
): ClosedTap[W] = { | ||
val sideViews: java.lang.Iterable[PCollectionView[_]] = sides.map(_.view).asJava | ||
|
||
val fn = new SortedBucketTransform.TransformFnWithSideInputContext[KeyType, W]() { | ||
|
@@ -197,7 +197,9 @@ object SortMergeTransform { | |
} | ||
val t = transform.via(fn, sideViews) | ||
sc.applyInternal(t) | ||
ClosedTap[Nothing](EmptyTap) | ||
|
||
val writeResult = sc.applyInternal(t) | ||
Comment on lines
199
to
+201
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Introduction of duplicated transform |
||
ClosedTap(SmbIO.tap(t.getFileOperations, writeResult).apply(sc)) | ||
} | ||
} | ||
|
||
|
@@ -213,11 +215,10 @@ object SortMergeTransform { | |
SideInputContext[_], | ||
SortedBucketTransform.SerializableConsumer[W] | ||
) => Unit | ||
): ClosedTap[Nothing] = { | ||
): ClosedTap[W] = { | ||
val data = read.parDo(new ViaTransformWithSideOutput(transformFn)) | ||
val testOutput = TestDataManager.getOutput(sc.testId.get) | ||
testOutput(SortedBucketIOUtil.testId(output))(data) | ||
ClosedTap[Nothing](EmptyTap) | ||
TestDataManager.getOutput(sc.testId.get)(SortedBucketIOUtil.testId(output)) | ||
ClosedTap(TapOf[W].saveForTest(data)) | ||
} | ||
} | ||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We could get around the signature change by adding a new method to SortedBucketIO.Write like:
but this seems overtly hacky to me - plus, we re-create a FileOperations instance for every bucket/shard combo
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
tbh it might be simplest to just make all these getters public 🤷♀️