-
Notifications
You must be signed in to change notification settings - Fork 2.4k
/
HoodiePipeline.java
263 lines (236 loc) · 10.1 KB
/
HoodiePipeline.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
/*
* 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.hudi.util;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieTableFactory;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.ResolvedCatalogTable;
import org.apache.flink.table.catalog.exceptions.TableNotExistException;
import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* A tool class to construct hoodie flink pipeline.
*
* <p>How to use ?</p>
* Method {@link #builder(String)} returns a pipeline builder. The builder
* can then define the hudi table columns, primary keys and partitions.
*
* <p>An example:</p>
* <pre>
* HoodiePipeline.Builder builder = HoodiePipeline.builder("myTable");
* DataStreamSink<?> sinkStream = builder
* .column("f0 int")
* .column("f1 varchar(10)")
* .column("f2 varchar(20)")
* .pk("f0,f1")
* .partition("f2")
* .sink(input, false);
* </pre>
*/
public class HoodiePipeline {
private static final Logger LOG = LogManager.getLogger(HoodiePipeline.class);
/**
* Returns the builder for hoodie pipeline construction.
*/
public static Builder builder(String tableName) {
return new Builder(tableName);
}
/**
* Builder for hudi source/sink pipeline construction.
*/
public static class Builder {
private final String tableName;
private final List<String> columns;
private final Map<String, String> options;
private String pk;
private List<String> partitions;
public Builder self() {
return this;
}
private Builder(String tableName) {
this.tableName = tableName;
this.columns = new ArrayList<>();
this.options = new HashMap<>();
this.partitions = new ArrayList<>();
}
/**
* Add a table column definition.
*
* @param column the column format should be in the form like 'f0 int'
*/
public Builder column(String column) {
this.columns.add(column);
return self();
}
/**
* Add primary keys.
*/
public Builder pk(String... pks) {
this.pk = String.join(",", pks);
return self();
}
/**
* Add partition fields.
*/
public Builder partition(String... partitions) {
this.partitions = new ArrayList<>(Arrays.asList(partitions));
return self();
}
/**
* Add a config option.
*/
public Builder option(ConfigOption<?> option, Object val) {
this.options.put(option.key(), val.toString());
return self();
}
public Builder option(String key, Object val) {
this.options.put(key, val.toString());
return self();
}
public Builder options(Map<String, String> options) {
this.options.putAll(options);
return self();
}
public DataStreamSink<?> sink(DataStream<RowData> input, boolean bounded) {
TableDescriptor tableDescriptor = getTableDescriptor();
return HoodiePipeline.sink(input, tableDescriptor.getTableId(), tableDescriptor.getResolvedCatalogTable(), bounded);
}
public TableDescriptor getTableDescriptor() {
EnvironmentSettings environmentSettings = EnvironmentSettings
.newInstance()
.build();
TableEnvironmentImpl tableEnv = TableEnvironmentImpl.create(environmentSettings);
String sql = getCreateHoodieTableDDL(this.tableName, this.columns, this.options, this.pk, this.partitions);
tableEnv.executeSql(sql);
String currentCatalog = tableEnv.getCurrentCatalog();
ResolvedCatalogTable catalogTable = null;
String defaultDatabase = null;
try {
Catalog catalog = tableEnv.getCatalog(currentCatalog).get();
defaultDatabase = catalog.getDefaultDatabase();
catalogTable = (ResolvedCatalogTable) catalog.getTable(new ObjectPath(defaultDatabase, this.tableName));
} catch (TableNotExistException e) {
throw new HoodieException("Create table " + this.tableName + " exception", e);
}
ObjectIdentifier tableId = ObjectIdentifier.of(currentCatalog, defaultDatabase, this.tableName);
return new TableDescriptor(tableId, catalogTable);
}
public DataStream<RowData> source(StreamExecutionEnvironment execEnv) {
TableDescriptor tableDescriptor = getTableDescriptor();
return HoodiePipeline.source(execEnv, tableDescriptor.tableId, tableDescriptor.getResolvedCatalogTable());
}
}
private static String getCreateHoodieTableDDL(
String tableName,
List<String> fields,
Map<String, String> options,
String pkField,
List<String> partitionField) {
StringBuilder builder = new StringBuilder();
builder.append("create table ").append(tableName).append("(\n");
for (String field : fields) {
builder.append(" ").append(field).append(",\n");
}
builder.append(" PRIMARY KEY(").append(pkField).append(") NOT ENFORCED\n")
.append(")\n");
if (!partitionField.isEmpty()) {
String partitons = partitionField
.stream()
.map(partitionName -> "`" + partitionName + "`")
.collect(Collectors.joining(","));
builder.append("PARTITIONED BY (").append(partitons).append(")\n");
}
builder.append("with (\n"
+ " 'connector' = 'hudi'");
options.forEach((k, v) -> builder.append(",\n")
.append(" '").append(k).append("' = '").append(v).append("'"));
builder.append("\n)");
return builder.toString();
}
/**
* low-level sink api for insert a datastream to hoodie table described by options and schema
* @param input The Input DataStream
* @param tablePath The tablePath to search a hoodie table in catalog
* @param catalogTable The catalog table to describe hoodie table schema
* @param isBouned The flag to indicate whether a batch execution mode
* @return The data stream sink return by insert a data stream to hoodie table
*/
private static DataStreamSink<?> sink(DataStream<RowData> input, ObjectIdentifier tablePath, ResolvedCatalogTable catalogTable, boolean isBouned) {
FactoryUtil.DefaultDynamicTableContext context = new FactoryUtil.DefaultDynamicTableContext(tablePath, catalogTable,
Configuration.fromMap(catalogTable.getOptions()), Thread.currentThread().getContextClassLoader(), false);
HoodieTableFactory hoodieTableFactory = new HoodieTableFactory();
return ((DataStreamSinkProvider) hoodieTableFactory.createDynamicTableSink(context)
.getSinkRuntimeProvider(new SinkRuntimeProviderContext(isBouned)))
.consumeDataStream(input);
}
/**
* low-level source api for read a hoodie table who described by options and schema as a datastream source
* @param execEnv The flink stream execute environment
* @param tablePath The tablePath to search a hoodie table in catalog
* @param catalogTable The catalog table to describe hoodie table schema
* @return the source data stream read from a hoodie table
*/
private static DataStream<RowData> source(StreamExecutionEnvironment execEnv, ObjectIdentifier tablePath, ResolvedCatalogTable catalogTable) {
FactoryUtil.DefaultDynamicTableContext context = new FactoryUtil.DefaultDynamicTableContext(tablePath, catalogTable,
Configuration.fromMap(catalogTable.getOptions()), Thread.currentThread().getContextClassLoader(), false);
HoodieTableFactory hoodieTableFactory = new HoodieTableFactory();
DataStreamScanProvider dataStreamScanProvider = (DataStreamScanProvider) ((ScanTableSource) hoodieTableFactory
.createDynamicTableSource(context))
.getScanRuntimeProvider(new ScanRuntimeProviderContext());
return dataStreamScanProvider.produceDataStream(execEnv);
}
/***
* a table descriptor for tableId and resolvedCatalogTable.
*/
public static class TableDescriptor {
private ObjectIdentifier tableId;
private ResolvedCatalogTable resolvedCatalogTable;
public TableDescriptor(ObjectIdentifier tableId, ResolvedCatalogTable resolvedCatalogTable) {
this.tableId = tableId;
this.resolvedCatalogTable = resolvedCatalogTable;
}
public ObjectIdentifier getTableId() {
return tableId;
}
public ResolvedCatalogTable getResolvedCatalogTable() {
return resolvedCatalogTable;
}
}
}