[ 
https://issues.apache.org/jira/browse/BEAM-9992?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17160331#comment-17160331
 ] 

Darshan Jani commented on BEAM-9992:
------------------------------------

I wrote a test and can see null as a key and null as value for GroupbyKey for 
bytes work fine.
If we notice, here in the test i need to use _NullableCoder.of_ to make sure 
nulls are handled correctly. But in our SQL case, we see it used only 
_ByteArrayCoder_ as a coder. I feel that is the problem.
What are your thoughts?

{code:java}
@Test
  @Category(NeedsRunner.class)
  public void testGroupByKeyOfBytesAndNulls() {

    byte[] bytes = {(byte) 0xff};
    byte[] bytes1 = {(byte) 0x80};
    byte[] bytes2 = {(byte) 0xc0};
    byte[] bytes3 = {(byte) 0xe0};
    byte[] bytes4 = {(byte) 0xf0};
    byte[] bytes5 = {(byte) 0xfe};
    byte[] bytes6 = {(byte) 0x80};
    List<KV<byte[], byte[]>> elems = Arrays.asList(
            KV.of(bytes, bytes1),
            KV.of(bytes, bytes2),
            KV.of(null, bytes3),
            KV.of(null, bytes4),
            KV.of(bytes5, null),
            KV.of(bytes5, bytes6));

    PCollection<KV<byte[], Iterable<byte[]>>> output = p.apply(
            Create.of(elems)
                    
.withCoder(KvCoder.of(NullableCoder.of(ByteArrayCoder.of()), 
NullableCoder.of(ByteArrayCoder.of()))))
            .apply(GroupByKey.create());

    PAssert.that(output).satisfies(
            i -> {
              for (KV<byte[], Iterable<byte[]>> item : i) {
                System.out.println(String.format("%s -> 
%s",item.getKey(),item.getValue()));
              }
              return null;
            });

    p.run();
  }
{code}


> Migrate BeamSQL's SET operators to Sets transforms
> --------------------------------------------------
>
>                 Key: BEAM-9992
>                 URL: https://issues.apache.org/jira/browse/BEAM-9992
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql, dsl-sql-zetasql
>            Reporter: Darshan Jani
>            Assignee: Darshan Jani
>            Priority: P2
>          Time Spent: 3h 20m
>  Remaining Estimate: 0h
>
> As par of [BEAM-9946|https://issues.apache.org/jira/browse/BEAM-9946] we have 
> new Sets transforms for intersect,union and except.
> This jira is to use them to remove existing Set operators in BeamSQL code.
> Tasks:
> # Remove: 
> [BeamSetOperatorRelBase.java|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java]
> # use SetFns transforms from
> ## 
> [BeamIntersectRel.java|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java]
> ## 
> [BeamMinusRel.java|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java]
> ## 
> [BeamMinusRel.java|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java]
> ## 
> [BeamUnionRel.java|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java]
> # 
> Remove:[BeamSetOperatorsTransforms.java|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java]



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

Reply via email to