SQL Write

Syntax

  1. INSERT { INTO | OVERWRITE } table_identifier [ part_spec ] [ column_list ] { value_expr | query };

For more information, please check the syntax document:

Spark INSERT Statement

INSERT INTO

Use INSERT INTO to apply records and changes to tables.

  1. INSERT INTO my_table SELECT ...

Overwriting the Whole Table

Use INSERT OVERWRITE to overwrite the whole unpartitioned table.

  1. INSERT OVERWRITE my_table SELECT ...

Overwriting a Partition

Use INSERT OVERWRITE to overwrite a partition.

  1. INSERT OVERWRITE my_table PARTITION (key1 = value1, key2 = value2, ...) SELECT ...

Dynamic Overwrite

Spark’s default overwrite mode is static partition overwrite. To enable dynamic overwritten you need to set the Spark session configuration spark.sql.sources.partitionOverwriteMode to dynamic

For example:

  1. CREATE TABLE my_table (id INT, pt STRING) PARTITIONED BY (pt);
  2. INSERT INTO my_table VALUES (1, 'p1'), (2, 'p2');
  3. -- Static overwrite (Overwrite the whole table)
  4. INSERT OVERWRITE my_table VALUES (3, 'p1');
  5. SELECT * FROM my_table;
  6. /*
  7. +---+---+
  8. | id| pt|
  9. +---+---+
  10. | 3| p1|
  11. +---+---+
  12. */
  13. -- Dynamic overwrite (Only overwrite pt='p1')
  14. SET spark.sql.sources.partitionOverwriteMode=dynamic;
  15. INSERT OVERWRITE my_table VALUES (3, 'p1');
  16. SELECT * FROM my_table;
  17. /*
  18. +---+---+
  19. | id| pt|
  20. +---+---+
  21. | 2| p2|
  22. | 3| p1|
  23. +---+---+
  24. */

Truncate tables

  1. TRUNCATE TABLE my_table;

Updating tables

spark supports update PrimitiveType and StructType, for example:

  1. -- Syntax
  2. UPDATE table_identifier SET column1 = value1, column2 = value2, ... WHERE condition;
  3. CREATE TABLE t (
  4. id INT,
  5. s STRUCT<c1: INT, c2: STRING>,
  6. name STRING)
  7. TBLPROPERTIES (
  8. 'primary-key' = 'id',
  9. 'merge-engine' = 'deduplicate'
  10. );
  11. -- you can use
  12. UPDATE t SET name = 'a_new' WHERE id = 1;
  13. UPDATE t SET s.c2 = 'a_new' WHERE s.c1 = 1;

Deleting from table

  1. DELETE FROM my_table WHERE currency = 'UNKNOWN';

Merging into table

Paimon currently supports Merge Into syntax in Spark 3+, which allow a set of updates, insertions and deletions based on a source table in a single commit.

  1. This only work with primary-key table.
  2. In update clause, to update primary key columns is not supported.
  3. WHEN NOT MATCHED BY SOURCE syntax is not supported.

Example: One

This is a simple demo that, if a row exists in the target table update it, else insert it.

  1. -- Here both source and target tables have the same schema: (a INT, b INT, c STRING), and a is a primary key.
  2. MERGE INTO target
  3. USING source
  4. ON target.a = source.a
  5. WHEN MATCHED THEN
  6. UPDATE SET *
  7. WHEN NOT MATCHED
  8. THEN INSERT *

Example: Two

This is a demo with multiple, conditional clauses.

  1. -- Here both source and target tables have the same schema: (a INT, b INT, c STRING), and a is a primary key.
  2. MERGE INTO target
  3. USING source
  4. ON target.a = source.a
  5. WHEN MATCHED AND target.a = 5 THEN
  6. UPDATE SET b = source.b + target.b -- when matched and meet the condition 1, then update b;
  7. WHEN MATCHED AND source.c > 'c2' THEN
  8. UPDATE SET * -- when matched and meet the condition 2, then update all the columns;
  9. WHEN MATCHED THEN
  10. DELETE -- when matched, delete this row in target table;
  11. WHEN NOT MATCHED AND c > 'c9' THEN
  12. INSERT (a, b, c) VALUES (a, b * 1.1, c) -- when not matched but meet the condition 3, then transform and insert this row;
  13. WHEN NOT MATCHED THEN
  14. INSERT * -- when not matched, insert this row without any transformation;

Streaming Write

Paimon currently supports Spark 3+ for streaming write.

Paimon Structured Streaming only supports the two append and complete modes.

  1. // Create a paimon table if not exists.
  2. spark.sql(s"""
  3. |CREATE TABLE T (k INT, v STRING)
  4. |TBLPROPERTIES ('primary-key'='a', 'bucket'='3')
  5. |""".stripMargin)
  6. // Here we use MemoryStream to fake a streaming source.
  7. val inputData = MemoryStream[(Int, String)]
  8. val df = inputData.toDS().toDF("k", "v")
  9. // Streaming Write to paimon table.
  10. val stream = df
  11. .writeStream
  12. .outputMode("append")
  13. .option("checkpointLocation", "/path/to/checkpoint")
  14. .format("paimon")
  15. .start("/path/to/paimon/sink/table")

Schema Evolution

Schema evolution is a feature that allows users to easily modify the current schema of a table to adapt to existing data, or new data that changes over time, while maintaining data integrity and consistency.

Paimon supports automatic schema merging of source data and current table data while data is being written, and uses the merged schema as the latest schema of the table, and it only requires configuring write.merge-schema.

  1. data.write
  2. .format("paimon")
  3. .mode("append")
  4. .option("write.merge-schema", "true")
  5. .save(location)

When enable write.merge-schema, Paimon can allow users to perform the following actions on table schema by default:

  • Adding columns
  • Up-casting the type of column(e.g. Int -> Long)

Paimon also supports explicit type conversions between certain types (e.g. String -> Date, Long -> Int), it requires an explicit configuration write.merge-schema.explicit-cast.

Schema evolution can be used in streaming mode at the same time.

  1. val inputData = MemoryStream[(Int, String)]
  2. inputData
  3. .toDS()
  4. .toDF("col1", "col2")
  5. .writeStream
  6. .format("paimon")
  7. .option("checkpointLocation", "/path/to/checkpoint")
  8. .option("write.merge-schema", "true")
  9. .option("write.merge-schema.explicit-cast", "true")
  10. .start(location)

Here list the configurations.

Scan ModeDescription
write.merge-schema
If true, merge the data schema and the table schema automatically before write data.
write.merge-schema.explicit-cast
If true, allow to merge data types if the two types meet the rules for explicit casting.