dxichen commented on code in PR #24276:
URL: https://github.com/apache/beam/pull/24276#discussion_r1034101967


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java:
##########
@@ -105,6 +106,38 @@ public static void createConfig(
     pipeline.traverseTopologically(visitor);
   }
 
+  /** Rewrite user store configs if there exists same state ids with multiple 
ParDos. */
+  public static void rewriteConfigWithMultiParDoStateId(
+      SamzaPipelineOptions options,
+      Map<String, String> multiParDoStateIdMap,
+      ConfigBuilder configBuilder) {
+    multiParDoStateIdMap.forEach(
+        (stateId, value) -> {
+          // rewrite single parDo state configs into multiple parDo state
+          String multiParDoStateId = String.join("-", stateId, value);
+          // replace old single parDo store configs with new storeId mapping 
appended with parDo
+          // name
+          configBuilder.remove("stores." + stateId + ".factory");
+          configBuilder.remove("stores." + stateId + ".key.serde");
+          configBuilder.remove("stores." + stateId + ".msg.serde");
+          configBuilder.remove("stores." + stateId + ".rocksdb.compression");
+          // put new config with multi pardo config
+          configBuilder.put(
+              "stores." + multiParDoStateId + ".factory",
+              
"org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory");

Review Comment:
   could we avoid using the classname since we are already including the 
package?



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java:
##########
@@ -140,9 +141,21 @@ public SamzaPipelineResult run(Pipeline pipeline) {
     LOG.info("Beam pipeline JSON graph:\n{}", jsonGraph);
 
     final Map<PValue, String> idMap = PViewToIdMapper.buildIdMap(pipeline);
+    /* Map of stateId to sanitized (remove whitespace and replace '-' with 
'_') PTransform name, used in multiple ParDos

Review Comment:
   nit: use /**



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java:
##########
@@ -149,37 +139,39 @@ static <K> Factory<K> createStateInternalsFactory(
       TaskContext context,
       SamzaPipelineOptions pipelineOptions,
       ExecutableStage executableStage) {
-
-    Set<String> stateIds =
+    // TODO: handle same stateIds in multiple ParDos for portable mode
+    Map<String, String> stateIds =
         executableStage.getUserStates().stream()
             .map(UserStateReference::localName)
-            .collect(Collectors.toSet());
+            .collect(Collectors.toMap(Function.identity(), 
Function.identity()));

Review Comment:
   Nit: could directly collect and remove the map step



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java:
##########
@@ -140,9 +141,21 @@ public SamzaPipelineResult run(Pipeline pipeline) {
     LOG.info("Beam pipeline JSON graph:\n{}", jsonGraph);
 
     final Map<PValue, String> idMap = PViewToIdMapper.buildIdMap(pipeline);
+    /* Map of stateId to sanitized (remove whitespace and replace '-' with 
'_') PTransform name, used in multiple ParDos

Review Comment:
   lets move this explanation to `rewriteConfigWithMultiPardoStateId`'s javadoc



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigContext.java:
##########
@@ -35,12 +34,17 @@ public class ConfigContext {
   private final Map<PValue, String> idMap;
   private AppliedPTransform<?, ?, ?> currentTransform;
   private final SamzaPipelineOptions options;
-  private final Set<String> stateIds;
+  private final Map<String, String> usedStateIdMap;
+  private final Map<String, String> multiParDoStateIdMap;

Review Comment:
   Lets rename this for clarity stateIdsToRewrite the value could also be the 
new state id. 



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java:
##########
@@ -129,18 +130,7 @@ static KeyValueStore<ByteArray, StateValue<?>> 
getBeamStore(TaskContext context)
    */
   static <K> Factory<K> createNonKeyedStateInternalsFactory(
       String id, TaskContext context, SamzaPipelineOptions pipelineOptions) {
-    return createStateInternalsFactory(id, null, context, pipelineOptions, 
Collections.emptySet());
-  }
-
-  static <K> Factory<K> createStateInternalsFactory(

Review Comment:
   Why did we remove creating for DoFnSignature?



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java:
##########
@@ -249,6 +252,19 @@ public String getTransformId() {
     return idGenerator.getId(getTransformFullName());
   }
 
+  /** Given a set of user stateIds and parDo name, return a stateId to storeId 
map. */
+  public Map<String, String> getStateIdToStoreIdMap(Set<String> stateIds, 
String escapedParDoName) {
+    final Map<String, String> storeIds = new HashMap<>();
+    stateIds.forEach(
+        stateId ->
+            storeIds.put(
+                stateId,
+                multiParDoStateIds.contains(stateId)
+                    ? String.join("-", stateId, escapedParDoName)

Review Comment:
   Let's create a helper method for this join, looks like it happens in a lot 
of place in the code.



-- 
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]

Reply via email to