-
Notifications
You must be signed in to change notification settings - Fork 2.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Core, Flink: Add task-type field to JSON serde of scan task / Add JSO…
…N serde for StaticDataTask. (#9728)
- Loading branch information
Showing
10 changed files
with
644 additions
and
28 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
/* | ||
* 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.iceberg; | ||
|
||
import com.fasterxml.jackson.core.JsonGenerator; | ||
import com.fasterxml.jackson.databind.JsonNode; | ||
import java.io.IOException; | ||
import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
import org.apache.iceberg.util.JsonUtil; | ||
|
||
class DataTaskParser { | ||
private static final String SCHEMA = "schema"; | ||
private static final String PROJECTED_SCHEMA = "projection"; | ||
private static final String METADATA_FILE = "metadata-file"; | ||
private static final String ROWS = "rows"; | ||
|
||
private DataTaskParser() {} | ||
|
||
static void toJson(StaticDataTask dataTask, JsonGenerator generator) throws IOException { | ||
Preconditions.checkArgument(dataTask != null, "Invalid data task: null"); | ||
Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); | ||
|
||
generator.writeFieldName(SCHEMA); | ||
SchemaParser.toJson(dataTask.schema(), generator); | ||
|
||
generator.writeFieldName(PROJECTED_SCHEMA); | ||
SchemaParser.toJson(dataTask.projectedSchema(), generator); | ||
|
||
generator.writeFieldName(METADATA_FILE); | ||
ContentFileParser.toJson(dataTask.metadataFile(), PartitionSpec.unpartitioned(), generator); | ||
|
||
Preconditions.checkArgument(dataTask.tableRows() != null, "Invalid data task: null table rows"); | ||
generator.writeArrayFieldStart(ROWS); | ||
for (StructLike row : dataTask.tableRows()) { | ||
SingleValueParser.toJson(dataTask.schema().asStruct(), row, generator); | ||
} | ||
|
||
generator.writeEndArray(); | ||
} | ||
|
||
static StaticDataTask fromJson(JsonNode jsonNode) { | ||
Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for data task: null"); | ||
Preconditions.checkArgument( | ||
jsonNode.isObject(), "Invalid JSON node for data task: non-object (%s)", jsonNode); | ||
|
||
Schema schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); | ||
Schema projectedSchema = SchemaParser.fromJson(JsonUtil.get(PROJECTED_SCHEMA, jsonNode)); | ||
DataFile metadataFile = | ||
(DataFile) | ||
ContentFileParser.fromJson( | ||
JsonUtil.get(METADATA_FILE, jsonNode), PartitionSpec.unpartitioned()); | ||
|
||
JsonNode rowsArray = JsonUtil.get(ROWS, jsonNode); | ||
Preconditions.checkArgument( | ||
rowsArray.isArray(), "Invalid JSON node for rows: non-array (%s)", rowsArray); | ||
|
||
StructLike[] rows = new StructLike[rowsArray.size()]; | ||
for (int i = 0; i < rowsArray.size(); ++i) { | ||
JsonNode rowNode = rowsArray.get(i); | ||
rows[i] = (StructLike) SingleValueParser.fromJson(schema.asStruct(), rowNode); | ||
} | ||
|
||
return new StaticDataTask(metadataFile, schema, projectedSchema, rows); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
105 changes: 105 additions & 0 deletions
105
core/src/main/java/org/apache/iceberg/ScanTaskParser.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,105 @@ | ||
/* | ||
* 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.iceberg; | ||
|
||
import com.fasterxml.jackson.core.JsonGenerator; | ||
import com.fasterxml.jackson.databind.JsonNode; | ||
import java.io.IOException; | ||
import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
import org.apache.iceberg.relocated.com.google.common.base.Strings; | ||
import org.apache.iceberg.util.JsonUtil; | ||
|
||
public class ScanTaskParser { | ||
private static final String TASK_TYPE = "task-type"; | ||
|
||
private enum TaskType { | ||
FILE_SCAN_TASK("file-scan-task"), | ||
DATA_TASK("data-task"); | ||
|
||
private final String value; | ||
|
||
TaskType(String value) { | ||
this.value = value; | ||
} | ||
|
||
public static TaskType fromTypeName(String value) { | ||
Preconditions.checkArgument( | ||
!Strings.isNullOrEmpty(value), "Invalid task type name: null or empty"); | ||
if (FILE_SCAN_TASK.typeName().equalsIgnoreCase(value)) { | ||
return FILE_SCAN_TASK; | ||
} else if (DATA_TASK.typeName().equalsIgnoreCase(value)) { | ||
return DATA_TASK; | ||
} else { | ||
throw new IllegalArgumentException("Unknown task type: " + value); | ||
} | ||
} | ||
|
||
public String typeName() { | ||
return value; | ||
} | ||
} | ||
|
||
private ScanTaskParser() {} | ||
|
||
public static String toJson(FileScanTask fileScanTask) { | ||
Preconditions.checkArgument(fileScanTask != null, "Invalid scan task: null"); | ||
return JsonUtil.generate(generator -> toJson(fileScanTask, generator), false); | ||
} | ||
|
||
public static FileScanTask fromJson(String json, boolean caseSensitive) { | ||
Preconditions.checkArgument(json != null, "Invalid JSON string for scan task: null"); | ||
return JsonUtil.parse(json, node -> fromJson(node, caseSensitive)); | ||
} | ||
|
||
private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) | ||
throws IOException { | ||
generator.writeStartObject(); | ||
|
||
if (fileScanTask instanceof StaticDataTask) { | ||
generator.writeStringField(TASK_TYPE, TaskType.DATA_TASK.typeName()); | ||
DataTaskParser.toJson((StaticDataTask) fileScanTask, generator); | ||
} else if (fileScanTask instanceof BaseFileScanTask | ||
|| fileScanTask instanceof BaseFileScanTask.SplitScanTask) { | ||
generator.writeStringField(TASK_TYPE, TaskType.FILE_SCAN_TASK.typeName()); | ||
FileScanTaskParser.toJson(fileScanTask, generator); | ||
} else { | ||
throw new UnsupportedOperationException( | ||
"Unsupported task type: " + fileScanTask.getClass().getCanonicalName()); | ||
} | ||
|
||
generator.writeEndObject(); | ||
} | ||
|
||
private static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { | ||
TaskType taskType = TaskType.FILE_SCAN_TASK; | ||
String taskTypeStr = JsonUtil.getStringOrNull(TASK_TYPE, jsonNode); | ||
if (null != taskTypeStr) { | ||
taskType = TaskType.fromTypeName(taskTypeStr); | ||
} | ||
|
||
switch (taskType) { | ||
case FILE_SCAN_TASK: | ||
return FileScanTaskParser.fromJson(jsonNode, caseSensitive); | ||
case DATA_TASK: | ||
return DataTaskParser.fromJson(jsonNode); | ||
default: | ||
throw new UnsupportedOperationException("Unsupported task type: " + taskType.typeName()); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.