You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by ni...@apache.org on 2023/03/31 07:16:12 UTC

[incubator-seatunnel-website] branch main updated: Add Blog (#221)

This is an automated email from the ASF dual-hosted git repository.

nielifeng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel-website.git


The following commit(s) were added to refs/heads/main by this push:
     new 1508cd9364 Add Blog (#221)
1508cd9364 is described below

commit 1508cd93640df2cdbc86434062b9337568ea9725
Author: lifeng <53...@users.noreply.github.com>
AuthorDate: Fri Mar 31 15:16:04 2023 +0800

    Add Blog (#221)
---
 ...ports_CDC_Writing_by_ClickHouse_Connector.md.md | 226 +++++++++++++++++++++
 ...a_in_Batches_420_Percent_Faster_than_GLUE.md.md | 127 ++++++++++++
 ...patible_Feature_Allows_ChatGPT_Automatic_Get.md |  82 ++++++++
 static/image/202303310331/01.png                   | Bin 0 -> 399285 bytes
 static/image/202303310331/contributors.png         | Bin 0 -> 110931 bytes
 static/image/202303311452/1.png                    | Bin 0 -> 14191 bytes
 static/image/202303311452/2.png                    | Bin 0 -> 2523680 bytes
 static/image/202303311452/3.png                    | Bin 0 -> 2686624 bytes
 static/image/202303311452/4.png                    | Bin 0 -> 3179688 bytes
 static/image/202303311452/5.png                    | Bin 0 -> 795111 bytes
 static/image/202303311452/6-1.png                  | Bin 0 -> 22397 bytes
 static/image/202303311452/6.png                    | Bin 0 -> 1184943 bytes
 static/image/202303311452/7.png                    | Bin 0 -> 2764296 bytes
 static/image/202303311452/8.png                    | Bin 0 -> 26607 bytes
 14 files changed, 435 insertions(+)

diff --git a/blog/2023-02-09-SeaTunnel_Now_Supports_CDC_Writing_by_ClickHouse_Connector.md.md b/blog/2023-02-09-SeaTunnel_Now_Supports_CDC_Writing_by_ClickHouse_Connector.md.md
new file mode 100755
index 0000000000..0269912352
--- /dev/null
+++ b/blog/2023-02-09-SeaTunnel_Now_Supports_CDC_Writing_by_ClickHouse_Connector.md.md
@@ -0,0 +1,226 @@
+# SeaTunnel now supports CDC (Capture Change Data) writing by ClickHouse Connector!
+
+Written by Wang Hailin, Apache SeaTunnel PPMC
+
+## Preface 
+Currently, SeaTunnel supports database change data capture (CDC https://github.com/apache/incubator-seatunnel/issues/3175), to transfer data changes to downstream systems in real time. SeaTunnel categorizes the captured data changes into the following 4 types: 
+- INSERT: Data insertion 
+- UPDATE_BEFORE: Historical value before data change 
+- UPDATE_AFTER: New value after data change 
+- DELETE: Data deletion 
+
+To handle the above data change operations, the Sink Connector needs to support writing behavior. This article will introduce how the ClickHouse Sink Connector supports writing these CDC types of data changes. 
+
+For CDC scenarios, the primary key is a necessary condition, so first, it needs to support the general requirements of INSERT, UPDATE, DELETE, etc. based on the primary key and ensure that the writing order is consistent with the CDC event order. In addition, considering the complexity of the data source in practice, it also needs to support UPSERT writing. Finally, according to the characteristics of ClickHouse itself, corresponding optimizations need to be made, such as UPDATE and DELE [...]
+
+## Overall design 
+
+The current ClickHouse Sink Connector is based on the JDBC Driver implementation, and a group of JDBC executors can be designed to encapsulate the processing of different types of data, making it convenient to switch or combine implementations based on actual scenarios and encapsulate implementation details. 
+
+JdbcBatchStatementExecutor is the top-level interface of the executor.
+
+```
+public interface JdbcBatchStatementExecutor extends AutoCloseable {
+
+    void prepareStatements(Connection connection) throws SQLException;
+
+    void addToBatch(SeaTunnelRow record) throws SQLException;
+
+    void executeBatch() throws SQLException;
+
+    void closeStatements() throws SQLException;
+
+    @Override
+    default void close() throws SQLException {
+        closeStatements();
+    }
+}
+```
+
+
+`JdbcBatchStatementExecutor` has the following implementation classes: 
+
+
+
+```
+SimpleBatchStatementExecutor // implements simple SQL Batch execution logic 
+InsertOrUpdateBatchStatementExecutor // implements INSERT, UPDATE update, also supports UPSERT mode 
+ReduceBufferedBatchStatementExecutor // memory accumulation, when refreshing to the database, the data change type (INSERT, UPDATE, DELETE) is distributed to the specific execution executor 
+```
+
+### Handling of cases where the primary key is not specified
+Currently, in CDC processing, the primary key is a necessary condition. If the Sink Connector is not specified in the primary key column configuration, it uses the append-only mode to write, calling `SimpleBatchStatementExecutor` directly.
+### CDC data process
+We divide the execution logic of data processing as follows: different data types enter the corresponding Executor and are finally transformed into their respective SQL statements for execution, and Jdbc Batch batching is used during this process.
+
+
+```
+CDC Event
+               /         \
+              /           \
+             /             \
+            /               \
+    DELETE Executor   INSERT OR UPDATE Executor
+                            /          \
+                           /            \
+                          /              \
+                         /                \
+                     INSERT Executor    UPDATE Executor
+```
+
+
+### Maintaining the Order of CDC Data
+CDC events are ordered, and writing must be processed in the order in which the events occur, otherwise data inconsistencies may occur. 
+
+In the previous logic, data of different types were distributed to their respective Executors and Jdbc Batch was used for batch submission to improve write performance, but categorizing batching can result in the order of submissions not being consistent with the CDC event order.
+
+We can add an execution barrier marker, when the processed data row is of the same type as the previous data row, it can be batched, if not, the previous batch is first flushed to the database, ensuring that the data write order is strictly consistent with the CDC event order.
+
+Example for `InsertOrUpdateBatchStatementExecutor`
+
+
+```
+public class InsertOrUpdateBatchStatementExecutor implements JdbcBatchStatementExecutor {
+    @Override
+    public void addToBatch(SeaTunnelRow record) throws SQLException {
+        boolean currentChangeFlag = hasInsert(record);
+        if (currentChangeFlag) {
+            if (preChangeFlag != null && !preChangeFlag) {
+                updateStatement.executeBatch();
+                updateStatement.clearBatch();
+            }
+            valueRowConverter.toExternal(record, insertStatement);
+            insertStatement.addBatch();
+        } else {
+            if (preChangeFlag != null && preChangeFlag) {
+                insertStatement.executeBatch();
+                insertStatement.clearBatch();
+            }
+            valueRowConverter.toExternal(record, updateStatement);
+            updateStatement.addBatch();
+        }
+        preChangeFlag = currentChangeFlag;
+        submitted = false;
+    }
+    
+    @Override
+    public void executeBatch() throws SQLException {
+        if (preChangeFlag != null) {
+            if (preChangeFlag) {
+                insertStatement.executeBatch();
+                insertStatement.clearBatch();
+            } else {
+                updateStatement.executeBatch();
+                updateStatement.clearBatch();
+            }
+        }
+        submitted = true;
+    }
+}
+```
+
+
+Of course, this will significantly slow down the batch processing, so we use `ReduceBufferedBatchStatementExecutor`to add a memory buffer layer, and when executing batch submissions, we distribute submissions to the database.
+
+Example for `ReduceBufferedBatchStatementExecutor`
+
+```
+public class ReduceBufferedBatchStatementExecutor implements JdbcBatchStatementExecutor {
+    private final LinkedHashMap<SeaTunnelRow, Pair<Boolean, SeaTunnelRow>> buffer = new LinkedHashMap<>();
+    
+    @Override
+    public void addToBatch(SeaTunnelRow record) throws SQLException {
+        buffer.put(record, ...);
+    }
+    
+    @Override
+    public void executeBatch() throws SQLException {
+        Boolean preChangeFlag = null;
+        Set<Map.Entry<SeaTunnelRow, Pair<Boolean, SeaTunnelRow>>> entrySet = buffer.entrySet();
+        for (Map.Entry<SeaTunnelRow, Pair<Boolean, SeaTunnelRow>> entry : entrySet) {
+            Boolean currentChangeFlag = entry.getValue().getKey();
+            if (currentChangeFlag) {
+                if (preChangeFlag != null && !preChangeFlag) {
+                    deleteExecutor.executeBatch();
+                }
+                insertOrUpdateExecutor.addToBatch(entry.getValue().getValue());
+            } else {
+                if (preChangeFlag != null && preChangeFlag) {
+                    insertOrUpdateExecutor.executeBatch();
+                }
+                deleteExecutor.addToBatch(entry.getKey());
+            }
+            preChangeFlag = currentChangeFlag;
+        }
+    
+        if (preChangeFlag != null) {
+            if (preChangeFlag) {
+                insertOrUpdateExecutor.executeBatch();
+            } else {
+                deleteExecutor.executeBatch();
+            }
+        }
+        buffer.clear();
+    }
+}
+```
+
+
+### Implementing a General UPSERT Write
+In `InsertOrUpdateBatchStatementExecutor`, you can configure to turn on UPSERT, when processing INSERT or UPDATE data types, it will first use the primary key to query the data row to see if it already exists and then decide to use INSERT or UPDATE SQL for writing. 
+
+*Note: This configuration is optional and will slow down the write speed, only opens when certain special scenarios are required.*
+
+Example for `InsertOrUpdateBatchStatementExecutor`
+
+```
+public class InsertOrUpdateBatchStatementExecutor implements JdbcBatchStatementExecutor {
+    @Override
+    public void addToBatch(SeaTunnelRow record) throws SQLException {
+        boolean currentChangeFlag = hasInsert(record);
+      ...
+    }
+
+    private boolean hasInsert(SeaTunnelRow record) throws SQLException {
+        if (upsertMode()) {
+            return !exist(keyExtractor.apply(record));
+        }
+        switch (record.getRowKind()) {
+            case INSERT:
+                return true;
+            case UPDATE_AFTER:
+                return false;
+            default:
+                throw new UnsupportedOperationException();
+        }
+    }
+    
+    private boolean exist(SeaTunnelRow pk) throws SQLException {
+        keyRowConverter.toExternal(pk, existStatement);
+        try (ResultSet resultSet = existStatement.executeQuery()) {
+            return resultSet.next();
+        }
+    }
+}
+```
+
+
+### Optimizing UPSERT for ReplacingMergeTree Engine
+
+The `ReplacingMergeTree` table engine can configure an `ORDER BY` field, and when executing the INSERT INTO statement, it covers the records with the same ORDER BY field. We can also utilize this feature to implement UPSERT.
+
+When the user writes to the `ReplacingMergeTree` table engine and the table's `ORDER BY` field is the same as the primary key field configured in the Sink Connector, both INSERT/UPDATE_AFTER data types are processed as INSERT to implement UPSERT.
+### Optimizing Updates for the MergeTree Engine
+
+DELETE and UPDATE are heavyweight operations in ClickHouse, but there is an experimental lightweight deletion (https://clickhouse.com/docs/en/sql-reference/statements/delete) for `MergeTree` engine, which performs better than the heavyweight deletion. We allow the user to configure the lightweight deletion.
+
+When the user writes to the `MergeTree` table engine and enables the lightweight deletion, we treat both DELETE/UPDATE_BEFORE data types as lightweight deletions, and treat both INSERT/UPDATE_AFTER data types as INSERTs, avoiding the UPDATE operation and using the lightweight deletion.
+
+## Related PR 
+- https://github.com/apache/incubator-seatunnel/pull/3653 
+
+Contribution to improving the related functions is welcomed, if you have any questions, please raise an issue on SeaTunnel GitHub (https://www.github.com/apache/incubator-seatunnel), and we will reply as soon as possible.
+
+## Reference 
+- https://clickhouse.com/docs/en/sql-reference/statements/delete 
+- https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/replacingmergetree
\ No newline at end of file
diff --git a/blog/2023-3-29-Performance_Test_Report_SeaTunnel_Synchronizes_Data_in_Batches_420_Percent_Faster_than_GLUE.md.md b/blog/2023-3-29-Performance_Test_Report_SeaTunnel_Synchronizes_Data_in_Batches_420_Percent_Faster_than_GLUE.md.md
new file mode 100755
index 0000000000..73d3158a32
--- /dev/null
+++ b/blog/2023-3-29-Performance_Test_Report_SeaTunnel_Synchronizes_Data_in_Batches_420_Percent_Faster_than_GLUE.md.md
@@ -0,0 +1,127 @@
+# Performance Test Report: SeaTunnel Synchronizes data in batches 420% Faster than GLUE!
+
+
+cover
+
+SeaTunnel Zeta has been officially released with the joint efforts of the community. After comparing the performance of SeaTunnel with DataX and Airbyte, we also compared the performance of SeaTunnel with the popular data synchronization tool AWS GLUE.
+
+The results showed that SeaTunnel batch syncs MySQL data to MySQL 420% faster than GLUE.
+
+To ensure the accuracy of the test, we took on the test under the same test environment: under the same resource conditions, we tested SeaTunnel and AWS GLUE to synchronize data from MySQL to MySQL in batches and compared the time required for the two tools.
+
+![1](/image/202303311452/1.png)
+
+
+We created a table in MySQL containing 31 fields, with the primary key selected as an incrementing ID, and all other fields generated randomly, without setting any indexes. The table creation statement is as follows:
+
+```plain
+create table test.type_source_table
+(
+    id                   int auto_increment
+        primary key,
+    f_binary             binary(64)          null,
+    f_blob               blob                null,
+    f_long_varbinary     mediumblob          null,
+    f_longblob           longblob            null,
+    f_tinyblob           tinyblob            null,
+    f_varbinary          varbinary(100)      null,
+    f_smallint           smallint            null,
+    f_smallint_unsigned  smallint unsigned   null,
+    f_mediumint          mediumint           null,
+    f_mediumint_unsigned mediumint unsigned  null,
+    f_int                int                 null,
+    f_int_unsigned       int unsigned        null,
+    f_integer            int                 null,
+    f_integer_unsigned   int unsigned        null,
+    f_bigint             bigint              null,
+    f_bigint_unsigned    bigint unsigned     null,
+    f_numeric            decimal             null,
+    f_decimal            decimal             null,
+    f_float              float               null,
+    f_double             double              null,
+    f_double_precision   double              null,
+    f_longtext           longtext            null,
+    f_mediumtext         mediumtext          null,
+    f_text               text                null,
+    f_tinytext           tinytext            null,
+    f_varchar            varchar(100)        null,
+    f_date               date                null,
+    f_datetime           datetime            null,
+    f_time               time                null,
+    f_timestamp          timestamp           null
+);
+```
+# SeaTunnel Task Configuration
+
+In SeaTunnel, we split the data according to the ID field and process it in multiple sub-tasks. Here is the configuration file for SeaTunnel:
+
+```plain
+env {
+    job.mode = "BATCH"
+    checkpoint.interval = 300000
+}
+source {
+    Jdbc {
+        url = "jdbc:mysql://XXX:3306/test"
+        driver = "com.mysql.cj.jdbc.Driver"
+        user = "root"
+        password = "password"
+        connection_check_timeout_sec = 100
+        query = "select id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint, f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer, f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double, f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime, f_time, f_timestamp from test"
+        partition_column = "id"
+        partition_num = 40
+        parallelism = 2
+    }
+}
+sink {
+Jdbc {
+          url = "jdbc:mysql://XXX:3306/test"
+         driver = "com.mysql.cj.jdbc.Driver" 
+        user = "root"
+        password = "password"
+         query = "insert into test_1 values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)"
+    }
+}
+```
+Under fixed JVM memory of 4G and parallelism of 2, SeaTunnel completed the synchronization in 1965 seconds. Based on this conclusion, we tested the speed of GLUE under the same memory and concurrency settings.
+# GLUE Task Configuration
+
+We created a MySQL-to-MySQL job as follows:
+
+![2](/image/202303311452/2.png)
+
+
+Configuration source connect with the target:
+
+![3](/image/202303311452/3.png)
+
+Job configuration:
+
+![4](/image/202303311452/4.png)
+
+
+![5](/image/202303311452/5.png)
+
+
+Adjust the memory: job parameters configuration
+
+![6](/image/202303311452/6-1.png)
+
+
+— conf spark.yarn.executor.memory=4g
+
+Under this configuration, GLUE took 8191 seconds to complete the synchronization.
+
+# Conclusion
+
+After comparing the best configurations, we conducted a more in-depth comparison for different memory sizes. The following chart shows the comparison results obtained through repeated testing under the same environment.
+
+![7](/image/202303311452/7.png)
+
+
+The unit is seconds.
+
+![8](/image/202303311452/8.png)
+
+Note: This comparison is based on SeaTunnel: commit ID f57b897, and we welcome to download and test it!
+
diff --git a/blog/2023-3-31-SeaTunnel_2_3_1_Released_Refactored_AI_Compatible_Feature_Allows_ChatGPT_Automatic_Get.md b/blog/2023-3-31-SeaTunnel_2_3_1_Released_Refactored_AI_Compatible_Feature_Allows_ChatGPT_Automatic_Get.md
new file mode 100755
index 0000000000..8255402013
--- /dev/null
+++ b/blog/2023-3-31-SeaTunnel_2_3_1_Released_Refactored_AI_Compatible_Feature_Allows_ChatGPT_Automatic_Get.md
@@ -0,0 +1,82 @@
+# SeaTunnel 2.3.1 is released! The refactored AI Compatible feature allows ChatGPT to automatically generate Connector code
+
+
+
+>*SeaTunnel version 2.3.1 was released recently. This is a high-profile release with many important function updates and optimizations.*
+>*At the level of programming user experience, the new version improves the stability of SeaTunnel Zeta and CI/CD; at the level of connectors, the new version implements 7+ new connectors and fixes existing commonly used connectors bugs, and improved security. The community refactored multiple underlying base classes and added an important feature, AI Compatible. With the optimized API, users can use ChatGPT 4.0 to quickly build the SaaS Connector they need.*
+# Major Feature update
+
+## 01 SeaTunnel Zeta
+
+The first version of the data integration engine-SeaTunnel Zeta is introduced in the SeaTunnel 2.3.0 release and has received feedback from numerous community users. In SeaTunnel version 2.3.1, we have fixed all the bugs reported by users, optimized the use of memory and threads, and greatly improved the stability of Zeta.
+
+In version 2.3.1, the community also added several new Zeta features, including a dedicated JVM parameter configuration file, client output of job monitoring information, Rest API for Zeta cluster information and job information, etc.
+
+At the checkpoint level, version 2.3.1 Zeta supports using OSS as checkpoint storage. It also supports savepoint running jobs and resuming jobs from savepoints.
+
+In addition, version 2.3.1 also adds a set of Zeta’s Rest API, which can be used to obtain the list of jobs running on Zeta, the status information of jobs, and the monitoring indicators of Zeta cluster nodes. For specific usage methods, please refer to **https:/ /seatunnel.apache.org/docs/seatunnel-engine/rest-api/**
+
+## 02 AI Compatible
+
+In SeaTunnel 2.3.1, the HTTP interface and related APIs are reconstructed, and the SaaS Connector-related API and Connector construction process are simplified according to the existing xGPT level capabilities so that ChatGPT 4.0 can directly generate SaaS Connectors and quickly generate various SaaS Connector interfaces. Under normal circumstances, the results obtained by this method are 95% similar to the code written by open-source contributors (see appendix).
+
+Of course, because ChatGPT4.0 will be updated in October 2021, it is necessary to provide some latest vectorized documents for the latest SaaS interface adaptation to have the latest interface adaptation. However, this refactored API and code framework allows users to generate Connectors more quickly and contribute to the open-source community, making the SeaTunnel interface more powerful.
+
+# Connector
+
+## 01 7+ new connectors
+
+While fixing the bugs of known connectors and optimizing the connectors, the community has added 7 new connectors including SAP HANA, Persistiq, TDEngine, SelectDB Cloud, Hbase, FieldMapper Transform, and SimpleSQL Transform.
+
+## 02 Reimplement SQL Transform
+
+Since the previous SQL Transform connector was defined based on Flink SQL and Spark SQL, SQL Transform cannot adapt to the execution of multiple engines, so we removed the SQL Transform function in version 2.3.0. In version 2.3.1, we reimplemented SQL Transform. SQL Transform is an API that does not depend on a task-specific execution engine and can perfectly run on three different engines: Flink/Spark/Zeta. Special thanks to contributor Ma Chengyuan (GitHub ID: rewerma) for leading and  [...]
+
+For the functions already supported by SQL Transform, please refer to [https://seatunnel.apache.org/docs/2.3.1/transform-v2/sql-functions](https://seatunnel.apache.org/docs/2.3.1/transform-v2/sql-functions)
+
+## 03 New SQL Server CDC
+
+At the CDC connector level, the community has newly added a SQL Server CDC connector, and made a lot of optimizations to MySQL CDC, improving the stability of MySQL CDC.
+
+## 04 Added CDC connector to output debezium-json format function
+
+In addition, version 2.3.1 also added the function of the CDC connector to output debezium-json format. Users can use MySQL CDC to read binlog and output data in debezium-json format to Kafka, so that users can create new synchronization tasks to read The data in debezium-json format in Kafka is synchronized to the target data source, or you can directly write other programs to read the data in debezium-json format in Kafka to perform some indicator calculations.
+
+# Safety
+
+Before version 2.3.1, users need to configure the database username, password, and other information in plain text in the config file, which may cause some security problems. In version 2.3.1, we added the configuration file encryption function, and users can fill in the encrypted database username, password, and other information in the config file. When the job is running, SeaTunnel will decrypt the content in the config file based on the default encryption and decryption algorithm. At [...]
+
+For how to use this function, please refer to [https://seatunnel.apache.org/docs/2.3.1/connector-v2/Config-Encryption-Decryption](https://seatunnel.apache.org/docs/2.3.1/connector-v2/Config-Encryption-Decryption)
+
+# Third-party engine support
+
+SeaTunnel version 2.3.1 supports Spark version 3.3, as well as Flink 1.14.6, Flink 1.15, Flink 1.16, and other versions, basically covering the mainstream versions of Spark and Flink.
+
+# Client
+
+The new version introduces an SPI for job configuration. Previously, only hocon json configuration files were supported. Now SPI is opened to the users to customize the format of job configuration files to meet different business system integration requirements.
+
+# Optimization
+
+SeaTunnel 2.1.3 version has made many important optimizations, including changes in core components, connector components, CI/CD, Zeta(ST-Engine), and E2E components, involving updating new functions, improving existing functions, and optimizing tests and deployment processes. Some notable changes include adding parallelism and column projection interfaces in Core API, introducing MySQL-CDC source factory in Connector-V2 and supporting only-once semantics for JDBC source connectors, impr [...]
+
+For a detailed list, see the Release Note [Improve] section.
+
+# Document
+
+In addition, the new version also has a series of updates to the documentation, including adding transform v2 documentation and some hints, as well as improving the documentation of various connectors.
+
+See the Release Note [Docs] section for details.
+
+Document address: [https://seatunnel.apache.org/versions/](https://seatunnel.apache.org/versions/)
+
+# Release Note
+
+[https://github.com/apache/incubator-seatunnel/blob/2.3.1/release-note.md](https://github.com/apache/incubator-seatunnel/blob/2.3.1/release-note.md)
+
+* Project address: [https://seatunnel.apache.org/](https://seatunnel.apache.org/)
+* Download address: [https://seatunnel.apache.org/download](https://seatunnel.apache.org/download)
+# Acknowledgement to the contributors
+
+![contributors](/image/202303310331/contributors.png)
+
diff --git a/static/image/202303310331/01.png b/static/image/202303310331/01.png
new file mode 100755
index 0000000000..f8f2540784
Binary files /dev/null and b/static/image/202303310331/01.png differ
diff --git a/static/image/202303310331/contributors.png b/static/image/202303310331/contributors.png
new file mode 100755
index 0000000000..a40e684730
Binary files /dev/null and b/static/image/202303310331/contributors.png differ
diff --git a/static/image/202303311452/1.png b/static/image/202303311452/1.png
new file mode 100755
index 0000000000..c9ff72ab99
Binary files /dev/null and b/static/image/202303311452/1.png differ
diff --git a/static/image/202303311452/2.png b/static/image/202303311452/2.png
new file mode 100755
index 0000000000..c9d0a72c09
Binary files /dev/null and b/static/image/202303311452/2.png differ
diff --git a/static/image/202303311452/3.png b/static/image/202303311452/3.png
new file mode 100755
index 0000000000..a3df7f7d9b
Binary files /dev/null and b/static/image/202303311452/3.png differ
diff --git a/static/image/202303311452/4.png b/static/image/202303311452/4.png
new file mode 100755
index 0000000000..0f3dafea90
Binary files /dev/null and b/static/image/202303311452/4.png differ
diff --git a/static/image/202303311452/5.png b/static/image/202303311452/5.png
new file mode 100755
index 0000000000..2fa941cece
Binary files /dev/null and b/static/image/202303311452/5.png differ
diff --git a/static/image/202303311452/6-1.png b/static/image/202303311452/6-1.png
new file mode 100755
index 0000000000..a1013136a9
Binary files /dev/null and b/static/image/202303311452/6-1.png differ
diff --git a/static/image/202303311452/6.png b/static/image/202303311452/6.png
new file mode 100755
index 0000000000..05bb49b028
Binary files /dev/null and b/static/image/202303311452/6.png differ
diff --git a/static/image/202303311452/7.png b/static/image/202303311452/7.png
new file mode 100755
index 0000000000..431d478580
Binary files /dev/null and b/static/image/202303311452/7.png differ
diff --git a/static/image/202303311452/8.png b/static/image/202303311452/8.png
new file mode 100755
index 0000000000..1b8fff97e2
Binary files /dev/null and b/static/image/202303311452/8.png differ