forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 5
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[java-extension-sql] Beam Sql doesn't honor aliases because of `BeamA…
…ggregateProjectMergeRule` optimization (apache#30902)
- Loading branch information
1 parent
e6b7a66
commit c218864
Showing
3 changed files
with
98 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
94 changes: 94 additions & 0 deletions
94
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlAliasTest
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
/* | ||
* 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.extensions.sql; | ||
|
||
import java.io.Serializable; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import org.apache.beam.sdk.coders.StringUtf8Coder; | ||
import org.apache.beam.sdk.schemas.Schema; | ||
import org.apache.beam.sdk.testing.PAssert; | ||
import org.apache.beam.sdk.testing.TestPipeline; | ||
import org.apache.beam.sdk.transforms.Create; | ||
import org.apache.beam.sdk.transforms.DoFn; | ||
import org.apache.beam.sdk.transforms.ParDo; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.Row; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.testcontainers.shaded.com.fasterxml.jackson.databind.MapperFeature; | ||
import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; | ||
|
||
public class BeamSqlAliasTest implements Serializable { | ||
|
||
@Rule public final transient TestPipeline pipeline = TestPipeline.create(); | ||
|
||
@Test | ||
public void testSqlWithAliasIsNotIgnoredWithOptimizers() { | ||
String ID = "id"; | ||
String EVENT = "event"; | ||
|
||
Schema inputType = Schema.builder().addStringField(ID).addStringField(EVENT).build(); | ||
|
||
String sql = | ||
"select event as event_name, count(*) as c\n" + "from PCOLLECTION\n" + "group by event"; | ||
|
||
List<Row> inputRows = | ||
TestUtils.RowsBuilder.of(inputType).addRows("123", "some_event").getRows(); | ||
|
||
PCollection<String> rowPCollection = | ||
pipeline | ||
.apply("boundedInput", Create.of(inputRows).withRowSchema(inputType)) | ||
.apply(SqlTransform.query(sql)) | ||
.apply( | ||
ParDo.of( | ||
new DoFn<Row, String>() { | ||
@DoFn.ProcessElement | ||
public void processElement(DoFn<Row, String>.ProcessContext c) | ||
throws Exception { | ||
ObjectMapper objectMapper = new ObjectMapper(); | ||
Map<String, Object> map = new HashMap<>(); | ||
|
||
for (int i = | ||
Objects.requireNonNull(c.element()).getSchema().getFields().size() | ||
- 1; | ||
i >= 0; | ||
i--) { | ||
Object value = Objects.requireNonNull(c.element()).getValue(i); | ||
Schema.Field field = | ||
Objects.requireNonNull(c.element()).getSchema().getField(i); | ||
map.put(field.getName(), value); | ||
} | ||
|
||
String json = | ||
objectMapper | ||
.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true) | ||
.writeValueAsString(map); | ||
c.output(json); | ||
} | ||
})) | ||
.setCoder(StringUtf8Coder.of()); | ||
|
||
// assert alias is kept | ||
PAssert.that(rowPCollection).containsInAnyOrder("{\"c\":1,\"event_name\":\"some_event\"}"); | ||
|
||
pipeline.run().waitUntilFinish(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters