forked from lakesoul-io/LakeSoul
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathLakeSoulMetadata.java
More file actions
236 lines (213 loc) · 10.6 KB
/
LakeSoulMetadata.java
File metadata and controls
236 lines (213 loc) · 10.6 KB
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
// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
//
// SPDX-License-Identifier: Apache-2.0
package com.facebook.presto.lakesoul;
import com.alibaba.fastjson.JSON;
import com.alibaba.fastjson.JSONObject;
import com.dmetasoul.lakesoul.meta.DBManager;
import com.dmetasoul.lakesoul.meta.DBUtil;
import com.dmetasoul.lakesoul.meta.entity.TableInfo;
import com.facebook.presto.lakesoul.handle.LakeSoulTableColumnHandle;
import com.facebook.presto.lakesoul.handle.LakeSoulTableHandle;
import com.facebook.presto.lakesoul.handle.LakeSoulTableLayoutHandle;
import com.facebook.presto.lakesoul.util.PrestoUtil;
import com.facebook.presto.spi.*;
import com.facebook.presto.spi.connector.ConnectorMetadata;
import com.google.common.collect.ImmutableList;
import org.apache.spark.sql.types.StructType;
import java.time.ZoneId;
import java.util.*;
import java.util.stream.Collectors;
import static com.facebook.presto.lakesoul.util.PrestoUtil.CDC_CHANGE_COLUMN;
public class LakeSoulMetadata implements ConnectorMetadata {
private final DBManager dbManager = new DBManager();
@Override
public List<String> listSchemaNames(ConnectorSession session) {
return dbManager.listNamespaces();
}
@Override
public List<SchemaTableName> listTables(ConnectorSession session, Optional<String> schemaName) {
String namespace = schemaName.orElse("default");
return dbManager.listTableNamesByNamespace(namespace)
.stream()
.map(name -> new SchemaTableName(namespace, name))
.collect(Collectors.toList());
}
@Override
public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) {
if (!listSchemaNames(session).contains(tableName.getSchemaName())) {
return null;
}
TableInfo
tableInfo =
dbManager.getTableInfoByNameAndNamespace(tableName.getTableName(), tableName.getSchemaName());
if (tableInfo == null) {
throw new RuntimeException("no such table: " + tableName);
}
LakeSoulTableHandle lakeSoulTableHandle = new LakeSoulTableHandle(
tableInfo.getTableId(),
tableName
);
return lakeSoulTableHandle;
}
@Override
public List<ConnectorTableLayoutResult> getTableLayouts(
ConnectorSession session, ConnectorTableHandle table,
Constraint<ColumnHandle> constraint,
Optional<Set<ColumnHandle>> desiredColumns) {
LakeSoulTableHandle tableHandle = (LakeSoulTableHandle) table;
TableInfo tableInfo = dbManager.getTableInfoByTableId(((LakeSoulTableHandle) table).getId());
DBUtil.TablePartitionKeys partitionKeys = DBUtil.parseTableInfoPartitions(tableInfo.getPartitions());
JSONObject properties = JSON.parseObject(tableInfo.getProperties());
StructType struct = (StructType) org.apache.spark.sql.types.DataType.fromJson(tableInfo.getTableSchema());
org.apache.arrow.vector.types.pojo.Schema arrowSchema =
org.apache.spark.sql.arrow.ArrowUtils.toArrowSchema(struct, ZoneId.of("UTC").toString());
HashMap<String, ColumnHandle> allColumns = new HashMap<>();
String cdcChangeColumn = properties.getString(CDC_CHANGE_COLUMN);
for (org.apache.arrow.vector.types.pojo.Field field : arrowSchema.getFields()) {
// drop cdc change column
if (cdcChangeColumn != null && field.getName().equals(cdcChangeColumn)) {
continue;
}
LakeSoulTableColumnHandle columnHandle =
new LakeSoulTableColumnHandle(tableHandle,
field.getName(),
PrestoUtil.convertToPrestoType(field.getType()));
allColumns.put(field.getName(), columnHandle);
}
ConnectorTableLayout layout = new ConnectorTableLayout(
new LakeSoulTableLayoutHandle(
tableHandle,
desiredColumns,
partitionKeys.primaryKeys,
partitionKeys.rangeKeys,
properties,
constraint.getSummary(),
allColumns
)
);
return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
}
@Override
public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) {
return new ConnectorTableLayout(handle);
}
@Override
public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) {
LakeSoulTableHandle handle = (LakeSoulTableHandle) table;
if (!listSchemaNames(session).contains(handle.getNames().getSchemaName())) {
return null;
}
TableInfo tableInfo = dbManager.getTableInfoByTableId(handle.getId());
if (tableInfo == null) {
throw new RuntimeException("no such table: " + handle.getNames());
}
JSONObject properties = JSON.parseObject(tableInfo.getProperties());
StructType struct = (StructType) org.apache.spark.sql.types.DataType.fromJson(tableInfo.getTableSchema());
org.apache.arrow.vector.types.pojo.Schema arrowSchema =
org.apache.spark.sql.arrow.ArrowUtils.toArrowSchema(struct, ZoneId.of("UTC").toString());
List<ColumnMetadata> columns = new LinkedList<>();
String cdcChangeColumn = properties.getString(CDC_CHANGE_COLUMN);
for (org.apache.arrow.vector.types.pojo.Field field : arrowSchema.getFields()) {
Map<String, Object> props = new HashMap<>();
for (Map.Entry<String, String> entry : field.getMetadata().entrySet()) {
props.put(entry.getKey(), entry.getValue());
}
// drop cdc change column
if (cdcChangeColumn != null && field.getName().equals(cdcChangeColumn)) {
continue;
}
ColumnMetadata columnMetadata = new ColumnMetadata(
field.getName(),
PrestoUtil.convertToPrestoType(field.getType()),
field.isNullable(),
"",
"",
false,
props
);
columns.add(columnMetadata);
}
return new ConnectorTableMetadata(
handle.getNames(),
columns,
properties,
Optional.of("")
);
}
@Override
public Map<String, ColumnHandle> getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) {
LakeSoulTableHandle table = (LakeSoulTableHandle) tableHandle;
TableInfo tableInfo = dbManager.getTableInfoByTableId(table.getId());
if (tableInfo == null) {
throw new RuntimeException("no such table: " + table.getNames());
}
JSONObject properties = JSON.parseObject(tableInfo.getProperties());
StructType struct = (StructType) org.apache.spark.sql.types.DataType.fromJson(tableInfo.getTableSchema());
org.apache.arrow.vector.types.pojo.Schema arrowSchema =
org.apache.spark.sql.arrow.ArrowUtils.toArrowSchema(struct, ZoneId.of("UTC").toString());
HashMap<String, ColumnHandle> map = new HashMap<>();
String cdcChangeColumn = properties.getString(CDC_CHANGE_COLUMN);
for (org.apache.arrow.vector.types.pojo.Field field : arrowSchema.getFields()) {
// drop cdc change column
if (cdcChangeColumn != null && field.getName().equals(cdcChangeColumn)) {
continue;
}
LakeSoulTableColumnHandle columnHandle =
new LakeSoulTableColumnHandle(table,
field.getName(),
PrestoUtil.convertToPrestoType(field.getType()));
map.put(field.getName(), columnHandle);
}
return map;
}
@Override
public ColumnMetadata getColumnMetadata(ConnectorSession session,
ConnectorTableHandle tableHandle,
ColumnHandle columnHandle) {
LakeSoulTableColumnHandle handle = (LakeSoulTableColumnHandle) columnHandle;
TableInfo tableInfo = dbManager.getTableInfoByTableId(handle.getTableHandle().getId());
if (tableInfo == null) {
throw new RuntimeException("no such table: " + handle.getTableHandle().getNames());
}
StructType struct = (StructType) org.apache.spark.sql.types.DataType.fromJson(tableInfo.getTableSchema());
org.apache.arrow.vector.types.pojo.Schema arrowSchema =
org.apache.spark.sql.arrow.ArrowUtils.toArrowSchema(struct, ZoneId.of("UTC").toString());
for (org.apache.arrow.vector.types.pojo.Field field : arrowSchema.getFields()) {
Map<String, Object> properties = new HashMap<>();
for (Map.Entry<String, String> entry : field.getMetadata().entrySet()) {
properties.put(entry.getKey(), entry.getValue());
}
if (field.getName().equals(handle.getColumnName())) {
return new ColumnMetadata(
field.getName(),
PrestoUtil.convertToPrestoType(field.getType()),
field.isNullable(),
"",
"",
false,
properties
);
}
}
throw new RuntimeException("no such column: " + handle.getColumnName());
}
@Override
public Map<SchemaTableName, List<ColumnMetadata>> listTableColumns(ConnectorSession session,
SchemaTablePrefix prefix) {
//prefix: lakesoul.default.table1
String schema = prefix.getSchemaName();
String tableNamePrefix = prefix.getTableName();
List<String> tableNames = dbManager.listTableNamesByNamespace(schema);
Map<SchemaTableName, List<ColumnMetadata>> results = new HashMap<>();
for (String tableName : tableNames) {
if (tableName.startsWith(tableNamePrefix)) {
SchemaTableName schemaTableName = new SchemaTableName(schema, tableName);
ConnectorTableHandle tableHandle = getTableHandle(session, schemaTableName);
ConnectorTableMetadata tableMetadata = getTableMetadata(session, tableHandle);
results.put(schemaTableName, tableMetadata.getColumns());
}
}
return results;
}
}