-
Notifications
You must be signed in to change notification settings - Fork 4.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
7327e45
commit 85c0e83
Showing
5 changed files
with
195 additions
and
7 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
74 changes: 74 additions & 0 deletions
74
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToJson.java
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,74 @@ | ||
/* | ||
* 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.transforms; | ||
|
||
import static org.apache.beam.sdk.util.RowJsonUtils.newObjectMapperWith; | ||
import static org.apache.beam.sdk.util.RowJsonUtils.rowToJson; | ||
|
||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import javax.annotation.Nullable; | ||
import org.apache.beam.sdk.annotations.Experimental; | ||
import org.apache.beam.sdk.schemas.Schema; | ||
import org.apache.beam.sdk.util.RowJsonSerializer; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.Row; | ||
|
||
/** | ||
* <i>Experimental</i> | ||
* | ||
* <p>Creates a {@link PTransform} that serializes UTF-8 JSON objects from a {@link Schema}-aware | ||
* PCollection (i.e. {@link PCollection#hasSchema()} returns true). JSON format is compatible with | ||
* {@link JsonToRow}. | ||
* | ||
* <p>For specifics of JSON serialization see {@link RowJsonSerializer}. | ||
*/ | ||
@Experimental | ||
public class ToJson<T> extends PTransform<PCollection<T>, PCollection<String>> { | ||
private transient volatile @Nullable ObjectMapper objectMapper; | ||
|
||
static <T> ToJson<T> of() { | ||
return new ToJson<T>(); | ||
} | ||
|
||
@Override | ||
public PCollection<String> expand(PCollection<T> rows) { | ||
Schema inputSchema = rows.getSchema(); | ||
SerializableFunction<T, Row> toRow = rows.getToRowFunction(); | ||
return rows.apply( | ||
ParDo.of( | ||
new DoFn<T, String>() { | ||
@ProcessElement | ||
public void processElement(ProcessContext context) { | ||
context.output( | ||
rowToJson(objectMapper(inputSchema), toRow.apply(context.element()))); | ||
} | ||
})); | ||
} | ||
|
||
private ObjectMapper objectMapper(Schema schema) { | ||
if (this.objectMapper == null) { | ||
synchronized (this) { | ||
if (this.objectMapper == null) { | ||
this.objectMapper = newObjectMapperWith(RowJsonSerializer.forSchema(schema)); | ||
} | ||
} | ||
} | ||
|
||
return this.objectMapper; | ||
} | ||
} |
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
95 changes: 95 additions & 0 deletions
95
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToJsonTest.java
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,95 @@ | ||
/* | ||
* 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.transforms; | ||
|
||
import com.google.auto.value.AutoValue; | ||
import java.io.Serializable; | ||
import org.apache.beam.sdk.schemas.AutoValueSchema; | ||
import org.apache.beam.sdk.schemas.Schema; | ||
import org.apache.beam.sdk.schemas.annotations.DefaultSchema; | ||
import org.apache.beam.sdk.testing.NeedsRunner; | ||
import org.apache.beam.sdk.testing.PAssert; | ||
import org.apache.beam.sdk.testing.TestPipeline; | ||
import org.apache.beam.sdk.testing.UsesSchema; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.Row; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.experimental.categories.Category; | ||
import org.junit.runner.RunWith; | ||
import org.junit.runners.JUnit4; | ||
|
||
/** Unit tests for {@link JsonToRow}. */ | ||
@RunWith(JUnit4.class) | ||
@Category(UsesSchema.class) | ||
public class ToJsonTest implements Serializable { | ||
|
||
@Rule public transient TestPipeline pipeline = TestPipeline.create(); | ||
|
||
@DefaultSchema(AutoValueSchema.class) | ||
@AutoValue | ||
abstract static class Person { | ||
public static Person of(String name, Integer height, Boolean knowsJavascript) { | ||
return new AutoValue_ToJsonTest_Person(name, height, knowsJavascript); | ||
} | ||
|
||
public abstract String getName(); | ||
|
||
public abstract Integer getHeight(); | ||
|
||
public abstract Boolean getKnowsJavascript(); | ||
} | ||
|
||
@Test | ||
@Category(NeedsRunner.class) | ||
public void testSerializesParseableJson() throws Exception { | ||
PCollection<Person> persons = | ||
pipeline.apply( | ||
"jsonPersons", | ||
Create.of( | ||
Person.of("person1", 80, true), | ||
Person.of("person2", 70, false), | ||
Person.of("person3", 60, true), | ||
Person.of("person4", 50, false), | ||
Person.of("person5", 40, true))); | ||
|
||
Schema personSchema = | ||
Schema.builder() | ||
.addStringField("name") | ||
.addInt32Field("height") | ||
.addBooleanField("knowsJavascript") | ||
.build(); | ||
|
||
PCollection<Row> personRows = | ||
persons.apply(ToJson.of()).apply(JsonToRow.withSchema(personSchema)); | ||
|
||
PAssert.that(personRows) | ||
.containsInAnyOrder( | ||
row(personSchema, "person1", 80, true), | ||
row(personSchema, "person2", 70, false), | ||
row(personSchema, "person3", 60, true), | ||
row(personSchema, "person4", 50, false), | ||
row(personSchema, "person5", 40, true)); | ||
|
||
pipeline.run(); | ||
} | ||
|
||
private Row row(Schema schema, Object... values) { | ||
return Row.withSchema(schema).addValues(values).build(); | ||
} | ||
} |
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