scwhittle commented on code in PR #36112:
URL: https://github.com/apache/beam/pull/36112#discussion_r2340513364
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java:
##########
@@ -190,21 +202,31 @@ public void processElement(
}
static class AssignShardFn<T> extends DoFn<T, KV<Integer, T>> {
- private int shard;
+ private int randomShard;
private @Nullable Integer numBuckets;
+ private boolean deterministicSharding;
- public AssignShardFn(@Nullable Integer numBuckets) {
+ public AssignShardFn(@Nullable Integer numBuckets, boolean
deterministicSharding) {
this.numBuckets = numBuckets;
+ this.deterministicSharding = deterministicSharding;
+ this.randomShard = 0;
}
@Setup
public void setup() {
- shard = ThreadLocalRandom.current().nextInt();
+ if (deterministicSharding) {
+ randomShard = ThreadLocalRandom.current().nextInt();
+ }
}
@ProcessElement
public void processElement(@Element T element, OutputReceiver<KV<Integer,
T>> r) {
- ++shard;
+ int shard = 0;
+ if (deterministicSharding && element != null) {
+ shard = element.hashCode();
Review Comment:
hashCode isn't deterministic across JVM restarts which we want this to be.
You can use FarmHash
https://guava.dev/releases/21.0/api/docs/com/google/common/hash/Hashing.html#farmHashFingerprint64--
To deterministically hash the right things you could have this dofn process
method take in ProcessContext and look at the offset or uniqueid from it. Since
that has some performance overhead I believe, you could have different dofn
impl for the random key assignment and the deterministic.
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java:
##########
@@ -190,21 +202,31 @@ public void processElement(
}
static class AssignShardFn<T> extends DoFn<T, KV<Integer, T>> {
- private int shard;
+ private int randomShard;
private @Nullable Integer numBuckets;
+ private boolean deterministicSharding;
- public AssignShardFn(@Nullable Integer numBuckets) {
+ public AssignShardFn(@Nullable Integer numBuckets, boolean
deterministicSharding) {
this.numBuckets = numBuckets;
+ this.deterministicSharding = deterministicSharding;
+ this.randomShard = 0;
}
@Setup
public void setup() {
- shard = ThreadLocalRandom.current().nextInt();
+ if (deterministicSharding) {
Review Comment:
!deterministic?
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java:
##########
@@ -158,7 +168,9 @@ public boolean getAllowDuplicates() {
@Override
public PCollection<T> expand(PCollection<T> input) {
return input
- .apply("Pair with random key", ParDo.of(new
AssignShardFn<>(numBuckets)))
Review Comment:
I think that changing the name means that update compatability woudl require
setting the transformnamemapping. Instead we could keep the name for the
previous behavior.
if (this.determinsticSharding) {
c = input.apply("Pair with deterministic key", ParDo.of())
} else {
c = input.apply("Pair with random key", ...)
}
return c.apply(Redistribute...)
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]