You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "jackye1995 (via GitHub)" <gi...@apache.org> on 2023/01/30 02:10:48 UTC

[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6571: Data: java api add GenericTaskWriter and add write demo to Doc.

jackye1995 commented on code in PR #6571:
URL: https://github.com/apache/iceberg/pull/6571#discussion_r1090118756


##########
docs/java-api.md:
##########
@@ -147,6 +147,59 @@ t.newAppend().appendFile(data).commit();
 t.commitTransaction();
 ```
 
+### WriteData
+
+The java api can write data into iceberg table.
+
+First write data to the data file, then submit the data file, the data you write will take effect in the table.
+
+For example, add 1000 pieces of data to the table.
+
+```java
+/**
+ * The structure of this table is the same as the demo table in java-api-quickstart
+ *
+ * Schema schema = new Schema(
+ *       Types.NestedField.required(1, "level", Types.StringType.get()),
+ *       Types.NestedField.required(2, "event_time", Types.TimestampType.withZone()),
+ *       Types.NestedField.required(3, "message", Types.StringType.get()),
+ *       Types.NestedField.optional(4, "call_stack", Types.ListType.ofRequired(5, Types.StringType.get()))
+ *   );
+ * PartitionSpec spec = PartitionSpec.builderFor(schema)
+ *       .hour("event_time")
+ *       .identity("level")
+ *       .build();
+ */
+
+GenericAppenderFactory appenderFactory = new GenericAppenderFactory(table.schema(), table.spec());
+int partitionId = 1, taskId = 1;
+FileFormat fileFormat = FileFormat.valueOf(table.properties().getOrDefault("write.format.default", "parquet").toUpperCase());
+OutputFileFactory outputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId).format(fileFormat).build();
+// TaskWriter write records into files. (the same is ok for unpartiton table)
+long targetFileSizeInBytes = 50L * 1024 * 1024;
+GenericTaskWriter<Record> genericTaskWriter = new GenericTaskWriter(table.spec(), fileFormat, appenderFactory, outputFileFactory, table.io(), targetFileSizeInBytes);
+
+GenericRecord genericRecord = GenericRecord.create(table.schema());
+// assume write 1000 records
+for (int i = 0; i < 1000; i++) {
+    GenericRecord record = genericRecord.copy();
+    record.setField("level",  i % 6 == 0 ? "error" : "info");
+    record.setField("event_time", OffsetDateTime.now());
+    record.setField("message", "Iceberg is a great table format");
+    record.setField("call_stack", Collections.singletonList("NullPointerException"));
+    genericTaskWriter.write(record);
+}
+// after the data file is written above,
+// the written data file is submitted to the metadata of the table through Table's Api.
+AppendFiles appendFiles = table.newAppend();
+for (DataFile dataFile : genericTaskWriter.dataFiles()) {
+    appendFiles.appendFile(dataFile);
+}
+// The submitted file generates a snapshot and submit it.
+Snapshot res = appendFiles.apply(); // Of course, you can skip apply to commit directly.

Review Comment:
   in that case I think we can remove this line to avoid confusion? Because the snapshot is not required to commit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org