|
| 1 | +/* |
| 2 | + * Copyright 2022 Bytedance Ltd. and/or its affiliates. |
| 3 | + * |
| 4 | + * Licensed under the Apache License, Version 2.0 (the "License"); |
| 5 | + * you may not use this file except in compliance with the License. |
| 6 | + * You may obtain a copy of the License at |
| 7 | + * |
| 8 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 12 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 13 | + * See the License for the specific language governing permissions and |
| 14 | + * limitations under the License. |
| 15 | + */ |
| 16 | + |
| 17 | +package com.bytedance.bitsail.connector.hadoop.format; |
| 18 | + |
| 19 | +import com.bytedance.bitsail.base.format.DeserializationSchema; |
| 20 | +import com.bytedance.bitsail.common.BitSailException; |
| 21 | +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; |
| 22 | +import com.bytedance.bitsail.common.model.ColumnInfo; |
| 23 | +import com.bytedance.bitsail.common.row.Row; |
| 24 | +import com.bytedance.bitsail.common.typeinfo.TypeInfo; |
| 25 | +import com.bytedance.bitsail.connector.hadoop.error.HadoopErrorCode; |
| 26 | +import com.bytedance.bitsail.connector.hadoop.option.HadoopReaderOptions; |
| 27 | + |
| 28 | +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; |
| 29 | +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; |
| 30 | +import org.apache.hadoop.hive.serde2.SerDeException; |
| 31 | +import org.apache.hadoop.hive.serde2.io.DateWritable; |
| 32 | +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; |
| 33 | +import org.apache.hadoop.hive.serde2.io.ShortWritable; |
| 34 | +import org.apache.hadoop.hive.serde2.io.TimestampWritable; |
| 35 | +import org.apache.hadoop.hive.serde2.objectinspector.StructField; |
| 36 | +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; |
| 37 | +import org.apache.hadoop.io.BooleanWritable; |
| 38 | +import org.apache.hadoop.io.ByteWritable; |
| 39 | +import org.apache.hadoop.io.BytesWritable; |
| 40 | +import org.apache.hadoop.io.DoubleWritable; |
| 41 | +import org.apache.hadoop.io.FloatWritable; |
| 42 | +import org.apache.hadoop.io.IntWritable; |
| 43 | +import org.apache.hadoop.io.LongWritable; |
| 44 | +import org.apache.hadoop.io.Text; |
| 45 | +import org.apache.hadoop.io.Writable; |
| 46 | +import org.apache.hadoop.mapred.JobConf; |
| 47 | + |
| 48 | +import java.util.List; |
| 49 | +import java.util.Properties; |
| 50 | +import java.util.stream.Collectors; |
| 51 | + |
| 52 | +public class HiveInputFormatDeserializationSchema implements DeserializationSchema<Writable, Row> { |
| 53 | + private final BitSailConfiguration deserializationConfiguration; |
| 54 | + private final TypeInfo<?>[] typeInfos; |
| 55 | + private final String[] fieldNames; |
| 56 | + private final StructObjectInspector inspector; |
| 57 | + public HiveInputFormatDeserializationSchema(BitSailConfiguration deserializationConfiguration, |
| 58 | + TypeInfo<?>[] typeInfos, |
| 59 | + String[] fieldNames) { |
| 60 | + |
| 61 | + this.deserializationConfiguration = deserializationConfiguration; |
| 62 | + this.typeInfos = typeInfos; |
| 63 | + this.fieldNames = fieldNames; |
| 64 | + |
| 65 | + List<ColumnInfo> columnInfos = deserializationConfiguration.get(HadoopReaderOptions.COLUMNS); |
| 66 | + Properties p = new Properties(); |
| 67 | + String columns = columnInfos.stream().map(ColumnInfo::getName).collect(Collectors.joining(",")); |
| 68 | + String columnsTypes = columnInfos.stream().map(ColumnInfo::getType).collect(Collectors.joining(":")); |
| 69 | + p.setProperty("columns", columns); |
| 70 | + p.setProperty("columns.types", columnsTypes); |
| 71 | + String inputFormatClass = deserializationConfiguration.get(HadoopReaderOptions.HADOOP_INPUT_FORMAT_CLASS); |
| 72 | + try { |
| 73 | + switch (inputFormatClass) { |
| 74 | + case "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat": { |
| 75 | + OrcSerde serde = new OrcSerde(); |
| 76 | + serde.initialize(new JobConf(), p); |
| 77 | + this.inspector = (StructObjectInspector) serde.getObjectInspector(); |
| 78 | + break; |
| 79 | + } |
| 80 | + case "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat": { |
| 81 | + ParquetHiveSerDe serde = new ParquetHiveSerDe(); |
| 82 | + serde.initialize(new JobConf(), p); |
| 83 | + this.inspector = (StructObjectInspector) serde.getObjectInspector(); |
| 84 | + break; |
| 85 | + } |
| 86 | + default: |
| 87 | + throw BitSailException.asBitSailException(HadoopErrorCode.UNSUPPORTED_ENCODING, "unsupported input format class: " + inputFormatClass); |
| 88 | + } |
| 89 | + } catch (SerDeException e) { |
| 90 | + throw BitSailException.asBitSailException(HadoopErrorCode.UNSUPPORTED_COLUMN_TYPE, "unsupported column information."); |
| 91 | + } |
| 92 | + } |
| 93 | + |
| 94 | + @Override |
| 95 | + public Row deserialize(Writable message) { |
| 96 | + int arity = fieldNames.length; |
| 97 | + List<? extends StructField> fields = inspector.getAllStructFieldRefs(); |
| 98 | + Row row = new Row(arity); |
| 99 | + for (int i = 0; i < arity; ++i) { |
| 100 | + Object writableData = inspector.getStructFieldData(message, fields.get(i)); |
| 101 | + row.setField(i, getWritableValue(writableData)); |
| 102 | + } |
| 103 | + return row; |
| 104 | + } |
| 105 | + |
| 106 | + @Override |
| 107 | + public boolean isEndOfStream(Row nextElement) { |
| 108 | + return false; |
| 109 | + } |
| 110 | + |
| 111 | + private Object getWritableValue(Object writable) { |
| 112 | + Object ret; |
| 113 | + |
| 114 | + if (writable == null) { |
| 115 | + ret = null; |
| 116 | + } else if (writable instanceof IntWritable) { |
| 117 | + ret = ((IntWritable) writable).get(); |
| 118 | + } else if (writable instanceof Text) { |
| 119 | + ret = writable.toString(); |
| 120 | + } else if (writable instanceof LongWritable) { |
| 121 | + ret = ((LongWritable) writable).get(); |
| 122 | + } else if (writable instanceof ByteWritable) { |
| 123 | + ret = ((ByteWritable) writable).get(); |
| 124 | + } else if (writable instanceof DateWritable) { |
| 125 | + ret = ((DateWritable) writable).get(); |
| 126 | + } else if (writable instanceof DoubleWritable) { |
| 127 | + ret = ((DoubleWritable) writable).get(); |
| 128 | + } else if (writable instanceof TimestampWritable) { |
| 129 | + ret = ((TimestampWritable) writable).getTimestamp(); |
| 130 | + } else if (writable instanceof FloatWritable) { |
| 131 | + ret = ((FloatWritable) writable).get(); |
| 132 | + } else if (writable instanceof BooleanWritable) { |
| 133 | + ret = ((BooleanWritable) writable).get(); |
| 134 | + } else if (writable instanceof BytesWritable) { |
| 135 | + BytesWritable bytesWritable = (BytesWritable) writable; |
| 136 | + byte[] bytes = bytesWritable.getBytes(); |
| 137 | + ret = new byte[bytesWritable.getLength()]; |
| 138 | + System.arraycopy(bytes, 0, ret, 0, bytesWritable.getLength()); |
| 139 | + } else if (writable instanceof HiveDecimalWritable) { |
| 140 | + ret = ((HiveDecimalWritable) writable).getHiveDecimal().bigDecimalValue(); |
| 141 | + } else if (writable instanceof ShortWritable) { |
| 142 | + ret = ((ShortWritable) writable).get(); |
| 143 | + } else { |
| 144 | + ret = writable.toString(); |
| 145 | + } |
| 146 | + return ret; |
| 147 | + } |
| 148 | +} |
0 commit comments