|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.seatunnel.connectors.seatunnel.hudi.source; |
| 19 | + |
| 20 | +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.CONF_FILES; |
| 21 | +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.KERBEROS_PRINCIPAL; |
| 22 | +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.KERBEROS_PRINCIPAL_FILE; |
| 23 | +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.TABLE_PATH; |
| 24 | +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.TABLE_TYPE; |
| 25 | +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.USE_KERBEROS; |
| 26 | + |
| 27 | +import org.apache.seatunnel.api.common.PrepareFailException; |
| 28 | +import org.apache.seatunnel.api.common.SeaTunnelContext; |
| 29 | +import org.apache.seatunnel.api.serialization.DefaultSerializer; |
| 30 | +import org.apache.seatunnel.api.serialization.Serializer; |
| 31 | +import org.apache.seatunnel.api.source.Boundedness; |
| 32 | +import org.apache.seatunnel.api.source.SeaTunnelSource; |
| 33 | +import org.apache.seatunnel.api.source.SourceReader; |
| 34 | +import org.apache.seatunnel.api.source.SourceSplitEnumerator; |
| 35 | +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; |
| 36 | +import org.apache.seatunnel.api.table.type.SeaTunnelRow; |
| 37 | +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; |
| 38 | +import org.apache.seatunnel.common.config.CheckConfigUtil; |
| 39 | +import org.apache.seatunnel.common.config.CheckResult; |
| 40 | +import org.apache.seatunnel.common.constants.PluginType; |
| 41 | +import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiPluginException; |
| 42 | +import org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil; |
| 43 | + |
| 44 | +import org.apache.seatunnel.shade.com.typesafe.config.Config; |
| 45 | + |
| 46 | +import com.google.auto.service.AutoService; |
| 47 | + |
| 48 | +import java.io.IOException; |
| 49 | + |
| 50 | +@AutoService(SeaTunnelSource.class) |
| 51 | +public class HudiSource implements SeaTunnelSource<SeaTunnelRow, HudiSourceSplit, HudiSourceState> { |
| 52 | + |
| 53 | + private SeaTunnelContext seaTunnelContext; |
| 54 | + |
| 55 | + private SeaTunnelRowType typeInfo; |
| 56 | + |
| 57 | + private String filePath; |
| 58 | + |
| 59 | + private String tablePath; |
| 60 | + |
| 61 | + private String confFiles; |
| 62 | + |
| 63 | + private boolean useKerberos = false; |
| 64 | + |
| 65 | + @Override |
| 66 | + public String getPluginName() { |
| 67 | + return "Hudi"; |
| 68 | + } |
| 69 | + |
| 70 | + @Override |
| 71 | + public void prepare(Config pluginConfig) { |
| 72 | + CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, TABLE_PATH, CONF_FILES); |
| 73 | + if (!result.isSuccess()) { |
| 74 | + throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); |
| 75 | + } |
| 76 | + // default hudi table tupe is cow |
| 77 | + // TODO: support hudi mor table |
| 78 | + // TODO: support Incremental Query and Read Optimized Query |
| 79 | + if (!"cow".equalsIgnoreCase(pluginConfig.getString(TABLE_TYPE))) { |
| 80 | + throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Do not support hudi mor table yet!"); |
| 81 | + } |
| 82 | + try { |
| 83 | + this.confFiles = pluginConfig.getString(CONF_FILES); |
| 84 | + this.tablePath = pluginConfig.getString(TABLE_PATH); |
| 85 | + if (CheckConfigUtil.isValidParam(pluginConfig, USE_KERBEROS)) { |
| 86 | + this.useKerberos = pluginConfig.getBoolean(USE_KERBEROS); |
| 87 | + if (this.useKerberos) { |
| 88 | + CheckResult kerberosCheckResult = CheckConfigUtil.checkAllExists(pluginConfig, KERBEROS_PRINCIPAL, KERBEROS_PRINCIPAL_FILE); |
| 89 | + if (!kerberosCheckResult.isSuccess()) { |
| 90 | + throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); |
| 91 | + } |
| 92 | + HudiUtil.initKerberosAuthentication(HudiUtil.getConfiguration(this.confFiles), pluginConfig.getString(KERBEROS_PRINCIPAL), pluginConfig.getString(KERBEROS_PRINCIPAL_FILE)); |
| 93 | + } |
| 94 | + } |
| 95 | + this.filePath = HudiUtil.getParquetFileByPath(this.confFiles, tablePath); |
| 96 | + if (this.filePath == null) { |
| 97 | + throw new HudiPluginException(String.format("%s has no parquet file, please check!", tablePath)); |
| 98 | + } |
| 99 | + // should read from config or read from hudi metadata( wait catlog done) |
| 100 | + this.typeInfo = HudiUtil.getSeaTunnelRowTypeInfo(this.confFiles, this.filePath); |
| 101 | + |
| 102 | + } catch (HudiPluginException | IOException e) { |
| 103 | + throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Prepare HudiSource error.", e); |
| 104 | + } |
| 105 | + |
| 106 | + } |
| 107 | + |
| 108 | + @Override |
| 109 | + public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { |
| 110 | + this.seaTunnelContext = seaTunnelContext; |
| 111 | + } |
| 112 | + |
| 113 | + @Override |
| 114 | + public SeaTunnelDataType<SeaTunnelRow> getProducedType() { |
| 115 | + return this.typeInfo; |
| 116 | + } |
| 117 | + |
| 118 | + @Override |
| 119 | + public SourceReader<SeaTunnelRow, HudiSourceSplit> createReader(SourceReader.Context readerContext) throws Exception { |
| 120 | + return new HudiSourceReader(this.confFiles, readerContext, typeInfo); |
| 121 | + } |
| 122 | + |
| 123 | + @Override |
| 124 | + public Boundedness getBoundedness() { |
| 125 | + // Only support Snapshot Query now. |
| 126 | + // After support Incremental Query and Read Optimized Query, we should supoort UNBOUNDED. |
| 127 | + // TODO: support UNBOUNDED |
| 128 | + return Boundedness.BOUNDED; |
| 129 | + } |
| 130 | + |
| 131 | + @Override |
| 132 | + public SourceSplitEnumerator<HudiSourceSplit, HudiSourceState> createEnumerator(SourceSplitEnumerator.Context<HudiSourceSplit> enumeratorContext) throws Exception { |
| 133 | + return new HudiSourceSplitEnumerator(enumeratorContext, tablePath, this.confFiles); |
| 134 | + } |
| 135 | + |
| 136 | + @Override |
| 137 | + public SourceSplitEnumerator<HudiSourceSplit, HudiSourceState> restoreEnumerator(SourceSplitEnumerator.Context<HudiSourceSplit> enumeratorContext, HudiSourceState checkpointState) throws Exception { |
| 138 | + return new HudiSourceSplitEnumerator(enumeratorContext, tablePath, this.confFiles, checkpointState); |
| 139 | + } |
| 140 | + |
| 141 | + @Override |
| 142 | + public Serializer<HudiSourceState> getEnumeratorStateSerializer() { |
| 143 | + return new DefaultSerializer<>(); |
| 144 | + } |
| 145 | +} |
0 commit comments