|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.sdk.io; |
| 19 | + |
| 20 | +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; |
| 21 | + |
| 22 | +import com.google.auto.service.AutoService; |
| 23 | +import java.util.Arrays; |
| 24 | +import java.util.List; |
| 25 | +import org.apache.beam.sdk.metrics.Counter; |
| 26 | +import org.apache.beam.sdk.metrics.Metrics; |
| 27 | +import org.apache.beam.sdk.schemas.NoSuchSchemaException; |
| 28 | +import org.apache.beam.sdk.schemas.Schema; |
| 29 | +import org.apache.beam.sdk.schemas.SchemaRegistry; |
| 30 | +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; |
| 31 | +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; |
| 32 | +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; |
| 33 | +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; |
| 34 | +import org.apache.beam.sdk.transforms.DoFn; |
| 35 | +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; |
| 36 | +import org.apache.beam.sdk.transforms.ParDo; |
| 37 | +import org.apache.beam.sdk.transforms.SerializableFunction; |
| 38 | +import org.apache.beam.sdk.transforms.SimpleFunction; |
| 39 | +import org.apache.beam.sdk.values.PCollection; |
| 40 | +import org.apache.beam.sdk.values.PCollectionRowTuple; |
| 41 | +import org.apache.beam.sdk.values.PCollectionTuple; |
| 42 | +import org.apache.beam.sdk.values.Row; |
| 43 | +import org.apache.beam.sdk.values.TupleTag; |
| 44 | +import org.apache.beam.sdk.values.TupleTagList; |
| 45 | +import org.slf4j.Logger; |
| 46 | +import org.slf4j.LoggerFactory; |
| 47 | + |
| 48 | +@AutoService(SchemaTransformProvider.class) |
| 49 | +public class TFRecordReadSchemaTransformProvider |
| 50 | + extends TypedSchemaTransformProvider<TFRecordReadSchemaTransformConfiguration> { |
| 51 | + private static final String IDENTIFIER = "beam:schematransform:org.apache.beam:tfrecord_read:v1"; |
| 52 | + private static final String OUTPUT = "output"; |
| 53 | + private static final String ERROR = "errors"; |
| 54 | + public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {}; |
| 55 | + public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {}; |
| 56 | + private static final Logger LOG = |
| 57 | + LoggerFactory.getLogger(TFRecordReadSchemaTransformProvider.class); |
| 58 | + |
| 59 | + /** Returns the expected {@link SchemaTransform} of the configuration. */ |
| 60 | + @Override |
| 61 | + protected SchemaTransform from(TFRecordReadSchemaTransformConfiguration configuration) { |
| 62 | + return new TFRecordReadSchemaTransform(configuration); |
| 63 | + } |
| 64 | + |
| 65 | + /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ |
| 66 | + @Override |
| 67 | + public String identifier() { |
| 68 | + return IDENTIFIER; |
| 69 | + } |
| 70 | + |
| 71 | + /** Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. */ |
| 72 | + @Override |
| 73 | + public List<String> outputCollectionNames() { |
| 74 | + return Arrays.asList(OUTPUT, ERROR); |
| 75 | + } |
| 76 | + |
| 77 | + /** |
| 78 | + * An implementation of {@link SchemaTransform} for TFRecord read jobs configured using {@link |
| 79 | + * TFRecordReadSchemaTransformConfiguration}. |
| 80 | + */ |
| 81 | + static class TFRecordReadSchemaTransform extends SchemaTransform { |
| 82 | + private final TFRecordReadSchemaTransformConfiguration configuration; |
| 83 | + |
| 84 | + TFRecordReadSchemaTransform(TFRecordReadSchemaTransformConfiguration configuration) { |
| 85 | + this.configuration = configuration; |
| 86 | + } |
| 87 | + |
| 88 | + public Row getConfigurationRow() { |
| 89 | + try { |
| 90 | + // To stay consistent with our SchemaTransform configuration naming conventions, |
| 91 | + // we sort lexicographically |
| 92 | + return SchemaRegistry.createDefault() |
| 93 | + .getToRowFunction(TFRecordReadSchemaTransformConfiguration.class) |
| 94 | + .apply(configuration) |
| 95 | + .sorted() |
| 96 | + .toSnakeCase(); |
| 97 | + } catch (NoSuchSchemaException e) { |
| 98 | + throw new RuntimeException(e); |
| 99 | + } |
| 100 | + } |
| 101 | + |
| 102 | + @Override |
| 103 | + public PCollectionRowTuple expand(PCollectionRowTuple input) { |
| 104 | + // Validate configuration parameters |
| 105 | + configuration.validate(); |
| 106 | + |
| 107 | + TFRecordIO.Read readTransform = |
| 108 | + TFRecordIO.read().withCompression(Compression.valueOf(configuration.getCompression())); |
| 109 | + |
| 110 | + String filePattern = configuration.getFilePattern(); |
| 111 | + if (filePattern != null) { |
| 112 | + readTransform = readTransform.from(filePattern); |
| 113 | + } |
| 114 | + if (!configuration.getValidate()) { |
| 115 | + readTransform = readTransform.withoutValidation(); |
| 116 | + } |
| 117 | + |
| 118 | + // Read TFRecord files into a PCollection of byte arrays. |
| 119 | + PCollection<byte[]> tfRecordValues = input.getPipeline().apply(readTransform); |
| 120 | + |
| 121 | + // Define the schema for the row |
| 122 | + final Schema schema = Schema.of(Schema.Field.of("record", Schema.FieldType.BYTES)); |
| 123 | + Schema errorSchema = ErrorHandling.errorSchemaBytes(); |
| 124 | + boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); |
| 125 | + |
| 126 | + SerializableFunction<byte[], Row> bytesToRowFn = getBytesToRowFn(schema); |
| 127 | + |
| 128 | + // Apply bytes to row fn |
| 129 | + PCollectionTuple outputTuple = |
| 130 | + tfRecordValues.apply( |
| 131 | + ParDo.of( |
| 132 | + new ErrorFn( |
| 133 | + "TFRecord-read-error-counter", bytesToRowFn, errorSchema, handleErrors)) |
| 134 | + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); |
| 135 | + |
| 136 | + PCollectionRowTuple outputRows = |
| 137 | + PCollectionRowTuple.of("output", outputTuple.get(OUTPUT_TAG).setRowSchema(schema)); |
| 138 | + |
| 139 | + // Error handling |
| 140 | + PCollection<Row> errorOutput = outputTuple.get(ERROR_TAG).setRowSchema(errorSchema); |
| 141 | + if (handleErrors) { |
| 142 | + outputRows = |
| 143 | + outputRows.and( |
| 144 | + checkArgumentNotNull(configuration.getErrorHandling()).getOutput(), errorOutput); |
| 145 | + } |
| 146 | + return outputRows; |
| 147 | + } |
| 148 | + } |
| 149 | + |
| 150 | + public static SerializableFunction<byte[], Row> getBytesToRowFn(Schema schema) { |
| 151 | + return new SimpleFunction<byte[], Row>() { |
| 152 | + @Override |
| 153 | + public Row apply(byte[] input) { |
| 154 | + return Row.withSchema(schema).addValues(input).build(); |
| 155 | + } |
| 156 | + }; |
| 157 | + } |
| 158 | + |
| 159 | + public static class ErrorFn extends DoFn<byte[], Row> { |
| 160 | + private final SerializableFunction<byte[], Row> valueMapper; |
| 161 | + private final Counter errorCounter; |
| 162 | + private Long errorsInBundle = 0L; |
| 163 | + private final boolean handleErrors; |
| 164 | + private final Schema errorSchema; |
| 165 | + |
| 166 | + public ErrorFn( |
| 167 | + String name, |
| 168 | + SerializableFunction<byte[], Row> valueMapper, |
| 169 | + Schema errorSchema, |
| 170 | + boolean handleErrors) { |
| 171 | + this.errorCounter = Metrics.counter(TFRecordReadSchemaTransformProvider.class, name); |
| 172 | + this.valueMapper = valueMapper; |
| 173 | + this.handleErrors = handleErrors; |
| 174 | + this.errorSchema = errorSchema; |
| 175 | + } |
| 176 | + |
| 177 | + @ProcessElement |
| 178 | + public void process(@DoFn.Element byte[] msg, MultiOutputReceiver receiver) { |
| 179 | + Row mappedRow = null; |
| 180 | + try { |
| 181 | + mappedRow = valueMapper.apply(msg); |
| 182 | + } catch (Exception e) { |
| 183 | + if (!handleErrors) { |
| 184 | + throw new RuntimeException(e); |
| 185 | + } |
| 186 | + errorsInBundle += 1; |
| 187 | + LOG.warn("Error while parsing the element", e); |
| 188 | + receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, msg, e)); |
| 189 | + } |
| 190 | + if (mappedRow != null) { |
| 191 | + receiver.get(OUTPUT_TAG).output(mappedRow); |
| 192 | + } |
| 193 | + } |
| 194 | + |
| 195 | + @FinishBundle |
| 196 | + public void finish(FinishBundleContext c) { |
| 197 | + errorCounter.inc(errorsInBundle); |
| 198 | + errorsInBundle = 0L; |
| 199 | + } |
| 200 | + } |
| 201 | +} |
0 commit comments