blob: 5c37b7c49b0be37390f1b77f4730f34fdab2dd02 [file] [log] [blame]
/*
* 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.mapreduce.translation;
import static com.google.common.base.Preconditions.checkNotNull;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.Serializable;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.util.common.ReflectHelpers;
/**
* Encapsulates the PipelineOptions in serialized form to ship them to the cluster.
*/
public class SerializedPipelineOptions implements Serializable {
private final byte[] serializedOptions;
/** Lazily initialized copy of deserialized options. */
private transient PipelineOptions pipelineOptions;
public SerializedPipelineOptions(PipelineOptions options) {
checkNotNull(options, "PipelineOptions must not be null.");
try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
createMapper().writeValue(baos, options);
this.serializedOptions = baos.toByteArray();
} catch (Exception e) {
throw new RuntimeException("Couldn't serialize PipelineOptions.", e);
}
}
public PipelineOptions getPipelineOptions() {
if (pipelineOptions == null) {
try {
pipelineOptions = createMapper().readValue(serializedOptions, PipelineOptions.class);
FileSystems.setDefaultPipelineOptions(pipelineOptions);
} catch (IOException e) {
throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e);
}
}
return pipelineOptions;
}
/**
* Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing
* for user specified configuration injection into the ObjectMapper. This supports user custom
* types on {@link PipelineOptions}.
*/
private static ObjectMapper createMapper() {
return new ObjectMapper().registerModules(
ObjectMapper.findModules(ReflectHelpers.findClassLoader()));
}
}