You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/04/26 20:38:01 UTC

[GitHub] [beam] brachi-wernick opened a new pull request, #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

brachi-wernick opened a new pull request, #15699:
URL: https://github.com/apache/beam/pull/15699

   This PR is a continuous work for https://github.com/apache/beam/pull/15510.
   
   Currently there are 2 Coders for Metadata: default one: `org.apache.beam.sdk.io.fs.MetadataCoder` and enhanced one `org.apache.beam.sdk.io.fs.MetadataCoderV2`, the last can also decode-encode `lastModifiedMillis` and it is done in a new coder in order to support backward compatibility.
   
   This will be hard to maintain, we will need to create a new coder for any new field that will be added to `Metadata`.
   
   So, as suggested in this comment: https://github.com/apache/beam/pull/15510#issuecomment-928390587, I came up with new generic coder : `MetadataDynamicCoder`.
   
   `MetadataDynamicCoder` can decode/encode any new fields added to `Metadata` by sending getter, setter and coder.
   
   For example creating coder for `lastModifiedMillis`:
   
   ``` java
    new MetadataDynamicCoder()
           .withCoderForField(
               VarLongCoder.of(),
               Metadata::lastModifiedMillis,
               Metadata.Builder::setLastModifiedMillis);
   ```
   
   I chose to get explicit getter/setter to avoid reflection which has bad impact on performance.
   


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


[GitHub] [beam] aaltay commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1125441386

   Could this be merged? Should it be closed?


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


[GitHub] [beam] brachi-wernick commented on a diff in pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
brachi-wernick commented on code in PR #15699:
URL: https://github.com/apache/beam/pull/15699#discussion_r858430584


##########
sdks/java/core/src/test/java/org/apache/beam/sdk/io/fs/MetadataDynamicCoderTest.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.fs;
+
+import java.nio.file.Path;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/** Tests for {@link org.apache.beam.sdk.io.fs.MetadataDynamicCoder}. */
+public class MetadataDynamicCoderTest {
+
+  @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testEncodeDecodeWithDefaultLastModifiedMills() throws Exception {

Review Comment:
   Kind problematic, since all fields now in Metadata are handled. except the one that I used in the test.
   
   It can work if I won't use the basic `MetadataCoder` in this dynamic coder, and work only with the fields coders I get in the list, But I think it will be mess to developers to assign all these basic fields for the dynamic coder, it is easy now, that they need to send only new/special fields and not all the basic fields.
   
   (coder now first does : `MatchResult.Metadata.Builder builder = V1_CODER.decodeBuilder(inStream);`
   which covers most of the fields in Metadata. and new fields will be covered with this coder list of {coder,getter,setter})
   



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


[GitHub] [beam] aaltay commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1118667789

   What is the next step on this PR?


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


[GitHub] [beam] pabloem closed pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
pabloem closed pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata
URL: https://github.com/apache/beam/pull/15699


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


[GitHub] [beam] johnjcasey commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1126076888

   Run Java Precommit


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


[GitHub] [beam] asf-ci commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1110226651

   Can one of the admins verify this patch?


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


[GitHub] [beam] pabloem commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
pabloem commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1110226754

   reopening PR to triggert tests to re-run


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


[GitHub] [beam] johnjcasey commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1126081289

   Looks like tests are failing. @brachi-wernick could you investigate and resolve the breaks?


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


[GitHub] [beam] asf-ci commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1110226650

   Can one of the admins verify this patch?


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


[GitHub] [beam] asf-ci commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1110226647

   Can one of the admins verify this patch?


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


[GitHub] [beam] brachi-wernick commented on a diff in pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
brachi-wernick commented on code in PR #15699:
URL: https://github.com/apache/beam/pull/15699#discussion_r858424608


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataDynamicCoder.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.fs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.BiConsumer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StructuredCoder;
+import org.apache.beam.sdk.transforms.SerializableBiConsumer;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+public class MetadataDynamicCoder extends StructuredCoder<MatchResult.Metadata> {
+
+  private static final MetadataCoder V1_CODER = MetadataCoder.of();
+
+  private List<Coder<?>> coders = new ArrayList<>();

Review Comment:
   Extracted into class in https://github.com/apache/beam/pull/15699/commits/420d4aa7705ac3a5766f08ab3b4247cd6c3330b6
   
   Regrading parallel list, do you mean stream.parallel? and decode/encode in parallel? I think it won't work since we iterate the stream sequentially. 



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


[GitHub] [beam] johnjcasey commented on a diff in pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on code in PR #15699:
URL: https://github.com/apache/beam/pull/15699#discussion_r858674035


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataDynamicCoder.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.fs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.BiConsumer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StructuredCoder;
+import org.apache.beam.sdk.transforms.SerializableBiConsumer;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+public class MetadataDynamicCoder extends StructuredCoder<MatchResult.Metadata> {
+
+  private static final MetadataCoder V1_CODER = MetadataCoder.of();
+
+  private List<Coder<?>> coders = new ArrayList<>();

Review Comment:
   Ah no, parallel list for me means when we have 2 or more lists, where listA.get(2) is logically associated with listB.get(2)



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


[GitHub] [beam] johnjcasey commented on a diff in pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on code in PR #15699:
URL: https://github.com/apache/beam/pull/15699#discussion_r858674352


##########
sdks/java/core/src/test/java/org/apache/beam/sdk/io/fs/MetadataDynamicCoderTest.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.fs;
+
+import java.nio.file.Path;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/** Tests for {@link org.apache.beam.sdk.io.fs.MetadataDynamicCoder}. */
+public class MetadataDynamicCoderTest {
+
+  @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testEncodeDecodeWithDefaultLastModifiedMills() throws Exception {

Review Comment:
   Sounds good



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


[GitHub] [beam] asf-ci commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1110226643

   Can one of the admins verify this patch?


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


[GitHub] [beam] github-actions[bot] commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1609431916

   This pull request has been closed due to lack of activity. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.


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


[GitHub] [beam] github-actions[bot] commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1598700561

   This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@beam.apache.org list. Thank you for your contributions.


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


[GitHub] [beam] github-actions[bot] commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1102618916

   This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@beam.apache.org list. Thank you for your contributions.


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


[GitHub] [beam] johnjcasey commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1109758189

   LGTM


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


[GitHub] [beam] johnjcasey commented on a diff in pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on code in PR #15699:
URL: https://github.com/apache/beam/pull/15699#discussion_r854363882


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataDynamicCoder.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.fs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.BiConsumer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StructuredCoder;
+import org.apache.beam.sdk.transforms.SerializableBiConsumer;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+public class MetadataDynamicCoder extends StructuredCoder<MatchResult.Metadata> {
+
+  private static final MetadataCoder V1_CODER = MetadataCoder.of();
+
+  private List<Coder<?>> coders = new ArrayList<>();

Review Comment:
   Do we want to use parallel lists? I would typically create a new pojo of {coder, getter, setter} and have a list of that instead, to avoid off by one errors if we ever modify this class, or need to add a new aspect to the dynamic coder that would warrant a new list



##########
sdks/java/core/src/test/java/org/apache/beam/sdk/io/fs/MetadataDynamicCoderTest.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.fs;
+
+import java.nio.file.Path;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/** Tests for {@link org.apache.beam.sdk.io.fs.MetadataDynamicCoder}. */
+public class MetadataDynamicCoderTest {
+
+  @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testEncodeDecodeWithDefaultLastModifiedMills() throws Exception {

Review Comment:
   Would it be possible to add a test that uses two dynamic coders, so we can verify that adding multiple coders stack as expected?



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


[GitHub] [beam] aaltay commented on pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #15699:
URL: https://github.com/apache/beam/pull/15699#issuecomment-1104177709

   @chamikaramj / @johnjcasey - folks could you please review or find a new reviewer?


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


[GitHub] [beam] github-actions[bot] closed pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #15699: [BEAM-13640][BEAM-12883] Add MetadataDynamicCoder to support encode-decode for new fields in Metatdata
URL: https://github.com/apache/beam/pull/15699


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