/*
* 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.runners.spark.translation;
import static org.junit.Assert.assertEquals;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonDeserializer;
import com.fasterxml.jackson.databind.JsonSerializer;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.auto.service.AutoService;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.CrashingRunner;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/**
* Tests for {@link SparkRuntimeContext}.
*/
@RunWith(JUnit4.class)
public class SparkRuntimeContextTest {
/** PipelineOptions used to test auto registration of Jackson modules. */
public interface JacksonIncompatibleOptions extends PipelineOptions {
JacksonIncompatible getJacksonIncompatible();
void setJacksonIncompatible(JacksonIncompatible value);
}
/** A Jackson {@link Module} to test auto-registration of modules. */
@AutoService(Module.class)
public static class RegisteredTestModule extends SimpleModule {
public RegisteredTestModule() {
super("RegisteredTestModule");
setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class);
}
}
/** A class which Jackson does not know how to serialize/deserialize. */
public static class JacksonIncompatible {
private final String value;
public JacksonIncompatible(String value) {
this.value = value;
}
}
/** A Jackson mixin used to add annotations to other classes. */
@JsonDeserialize(using = JacksonIncompatibleDeserializer.class)
@JsonSerialize(using = JacksonIncompatibleSerializer.class)
public static final class JacksonIncompatibleMixin {}
/** A Jackson deserializer for {@link JacksonIncompatible}. */
public static class JacksonIncompatibleDeserializer extends
JsonDeserializer<JacksonIncompatible> {
@Override
public JacksonIncompatible deserialize(JsonParser jsonParser,
DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
return new JacksonIncompatible(jsonParser.readValueAs(String.class));
}
}
/** A Jackson serializer for {@link JacksonIncompatible}. */
public static class JacksonIncompatibleSerializer extends JsonSerializer<JacksonIncompatible> {
@Override
public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator,
SerializerProvider serializerProvider) throws IOException, JsonProcessingException {
jsonGenerator.writeString(jacksonIncompatible.value);
}
}
@Test
public void testSerializingPipelineOptionsWithCustomUserType() throws Exception {
PipelineOptions options = PipelineOptionsFactory.fromArgs("--jacksonIncompatible=\"testValue\"")
.as(JacksonIncompatibleOptions.class);
options.setRunner(CrashingRunner.class);
Pipeline p = Pipeline.create(options);
SparkRuntimeContext context = new SparkRuntimeContext(p, options);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try (ObjectOutputStream outputStream = new ObjectOutputStream(baos)) {
outputStream.writeObject(context);
}
try (ObjectInputStream inputStream =
new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray()))) {
SparkRuntimeContext copy = (SparkRuntimeContext) inputStream.readObject();
assertEquals("testValue",
copy.getPipelineOptions().as(JacksonIncompatibleOptions.class)
.getJacksonIncompatible().value);
}
}
}