beam-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (Jira)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-12400) Improve MongoDBIO for beam - add update capability
Date Tue, 31 Aug 2021 18:12:00 GMT

     [ https://issues.apache.org/jira/browse/BEAM-12400?focusedWorklogId=644557&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-644557
]

ASF GitHub Bot logged work on BEAM-12400:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 31/Aug/21 18:11
            Start Date: 31/Aug/21 18:11
    Worklog Time Spent: 10m 
      Work Description: pabloem commented on a change in pull request #14927:
URL: https://github.com/apache/beam/pull/14927#discussion_r699563891



##########
File path: sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateField.java
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.mongodb;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@Experimental(Kind.SOURCE_SINK)
+@AutoValue
+public abstract class UpdateField implements Serializable {
+
+  abstract @Nullable String updateOperator();
+
+  abstract @Nullable String sourceField();
+
+  abstract @Nullable String destField();
+
+  private static Builder builder() {
+    return new AutoValue_UpdateField.Builder().setSourceField(null);
+  }
+
+  abstract UpdateField.Builder toBuilder();
+
+  public static UpdateField create() {
+    return builder().build();
+  }
+
+  @AutoValue.Builder
+  abstract static class Builder {
+    abstract UpdateField.Builder setUpdateOperator(@Nullable String updateOperator);
+
+    abstract UpdateField.Builder setSourceField(@Nullable String sourceField);
+
+    abstract UpdateField.Builder setDestField(@Nullable String destField);
+
+    abstract UpdateField build();
+  }
+
+  /** Sets the limit of documents to find. */
+  public UpdateField fullUpdate(String updateOperator, String destField) {
+    return toBuilder().setUpdateOperator(updateOperator).setDestField(destField).build();
+  }

Review comment:
       In this case, we're inserting the full record into one field of the destination, right?
   
   e.g.:
   original
   ```
   {
     "key": "mykey1",
     "value1": "myvalue1",
     "value2": ["1", "2"],
     "value3": "thisvalue"
   }
   ```
   
   newvalue
   ```
   {
     "key": "mykey1",
     "value1": "myvalue1UPD",
     "value2": ["1", "2", "3", "4"],
     "value3": "thisvalueISUPDATED"
   }
   ```
   
   Given configuration `UpdateField.fullUpdate("$push", "value4")`:
   result
   ```
   {
     "key": "mykey1",
     "value1": "myvalue1",
     "value2": ["1", "2"],
     "value3": "thisvalue"
     "value4": {
       "key": "mykey1",
       "value1": "myvalue1UPD",
       "value2": ["1", "2", "3", "4"],
       "value3": "thisvalueISUPDATED"
     }
   }
   ```
   
   Is this correct / intended?




-- 
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: github-unsubscribe@beam.apache.org

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


Issue Time Tracking
-------------------

            Worklog Id:     (was: 644557)
    Remaining Estimate: 162h 20m  (was: 162.5h)
            Time Spent: 5h 40m  (was: 5.5h)

> Improve MongoDBIO for beam - add update capability
> --------------------------------------------------
>
>                 Key: BEAM-12400
>                 URL: https://issues.apache.org/jira/browse/BEAM-12400
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-mongodb
>            Reporter: Paresh Saraf
>            Assignee: Paresh Saraf
>            Priority: P2
>   Original Estimate: 168h
>          Time Spent: 5h 40m
>  Remaining Estimate: 162h 20m
>
> Right now mongodbio supports only inserts/overwrites to a collections. In many cases
it will be usually updating an existing document: setting a field or pushing into an array.
BulkUpdate capability to be added as part part of MongoDBIo->Write



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Mime
View raw message