├── .asf.yaml ├── .github ├── ISSUE_TEMPLATE │ ├── bug_report.yml │ ├── config.yml │ ├── enhancement.yml │ └── feature-request.yml ├── PULL_REQUEST_TEMPLATE.md └── workflows │ ├── approve-label-trigger.yml │ ├── approve-label.yml │ ├── build-connector.yml │ ├── checkstyle.yaml │ ├── license-eyes.yml │ ├── run-e2ecase.yml │ └── run-itcase.yml ├── .gitignore ├── .licenserc.yaml ├── .mvn └── wrapper │ ├── MavenWrapperDownloader.java │ └── maven-wrapper.properties ├── CODE_OF_CONDUCT.md ├── CONTRIBUTING.md ├── CONTRIBUTING_CN.md ├── LICENSE.txt ├── NOTICE.txt ├── README.md ├── custom_env.sh.tpl ├── env.sh ├── flink-doris-connector ├── build.sh ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ ├── doris │ │ │ └── flink │ │ │ │ ├── backend │ │ │ │ └── BackendClient.java │ │ │ │ ├── catalog │ │ │ │ ├── DorisCatalog.java │ │ │ │ ├── DorisCatalogFactory.java │ │ │ │ ├── DorisCatalogOptions.java │ │ │ │ ├── DorisTypeMapper.java │ │ │ │ └── doris │ │ │ │ │ ├── DataModel.java │ │ │ │ │ ├── DorisSchemaFactory.java │ │ │ │ │ ├── DorisSystem.java │ │ │ │ │ ├── DorisType.java │ │ │ │ │ ├── FieldSchema.java │ │ │ │ │ └── TableSchema.java │ │ │ │ ├── cfg │ │ │ │ ├── ConfigurationOptions.java │ │ │ │ ├── DorisConnectionOptions.java │ │ │ │ ├── DorisExecutionOptions.java │ │ │ │ ├── DorisLookupOptions.java │ │ │ │ ├── DorisOptions.java │ │ │ │ ├── DorisReadOptions.java │ │ │ │ └── DorisStreamOptions.java │ │ │ │ ├── connection │ │ │ │ ├── JdbcConnectionProvider.java │ │ │ │ └── SimpleJdbcConnectionProvider.java │ │ │ │ ├── datastream │ │ │ │ └── DorisSourceFunction.java │ │ │ │ ├── deserialization │ │ │ │ ├── DorisDeserializationSchema.java │ │ │ │ ├── RowDataDeserializationSchema.java │ │ │ │ ├── SimpleListDeserializationSchema.java │ │ │ │ └── converter │ │ │ │ │ └── DorisRowConverter.java │ │ │ │ ├── exception │ │ │ │ ├── ConnectedFailedException.java │ │ │ │ ├── CopyLoadException.java │ │ │ │ ├── CreateTableException.java │ │ │ │ ├── DorisBatchLoadException.java │ │ │ │ ├── DorisException.java │ │ │ │ ├── DorisInternalException.java │ │ │ │ ├── DorisRuntimeException.java │ │ │ │ ├── DorisSchemaChangeException.java │ │ │ │ ├── DorisSystemException.java │ │ │ │ ├── IllegalArgumentException.java │ │ │ │ ├── LabelAlreadyExistsException.java │ │ │ │ ├── ShouldNeverHappenException.java │ │ │ │ └── StreamLoadException.java │ │ │ │ ├── lookup │ │ │ │ ├── DorisJdbcLookupReader.java │ │ │ │ ├── DorisLookupReader.java │ │ │ │ ├── ExecutionPool.java │ │ │ │ ├── Get.java │ │ │ │ ├── GetAction.java │ │ │ │ ├── LookupMetrics.java │ │ │ │ ├── LookupSchema.java │ │ │ │ ├── Record.java │ │ │ │ ├── RecordKey.java │ │ │ │ └── Worker.java │ │ │ │ ├── rest │ │ │ │ ├── PartitionDefinition.java │ │ │ │ ├── RestService.java │ │ │ │ ├── SchemaUtils.java │ │ │ │ └── models │ │ │ │ │ ├── BackendV2.java │ │ │ │ │ ├── Field.java │ │ │ │ │ ├── QueryPlan.java │ │ │ │ │ ├── RespContent.java │ │ │ │ │ ├── Schema.java │ │ │ │ │ └── Tablet.java │ │ │ │ ├── serialization │ │ │ │ ├── Routing.java │ │ │ │ └── RowBatch.java │ │ │ │ ├── sink │ │ │ │ ├── BackendUtil.java │ │ │ │ ├── DorisAbstractCommittable.java │ │ │ │ ├── DorisCommittable.java │ │ │ │ ├── DorisCommittableSerializer.java │ │ │ │ ├── DorisSink.java │ │ │ │ ├── EscapeHandler.java │ │ │ │ ├── HttpGetWithEntity.java │ │ │ │ ├── HttpPutBuilder.java │ │ │ │ ├── HttpUtil.java │ │ │ │ ├── LoadStatus.java │ │ │ │ ├── ResponseUtil.java │ │ │ │ ├── batch │ │ │ │ │ ├── BatchBufferHttpEntity.java │ │ │ │ │ ├── BatchBufferStream.java │ │ │ │ │ ├── BatchRecordBuffer.java │ │ │ │ │ ├── DorisBatchSink.java │ │ │ │ │ ├── DorisBatchStreamLoad.java │ │ │ │ │ ├── DorisBatchWriter.java │ │ │ │ │ └── RecordWithMeta.java │ │ │ │ ├── committer │ │ │ │ │ └── DorisCommitter.java │ │ │ │ ├── copy │ │ │ │ │ ├── BackoffAndRetryUtils.java │ │ │ │ │ ├── BatchRecordBuffer.java │ │ │ │ │ ├── BatchStageLoad.java │ │ │ │ │ ├── CopyCommittableSerializer.java │ │ │ │ │ ├── CopySQLBuilder.java │ │ │ │ │ ├── DorisCopyCommittable.java │ │ │ │ │ ├── DorisCopyCommitter.java │ │ │ │ │ ├── DorisCopyWriter.java │ │ │ │ │ ├── HttpPostBuilder.java │ │ │ │ │ └── models │ │ │ │ │ │ ├── BaseResponse.java │ │ │ │ │ │ └── CopyIntoResp.java │ │ │ │ ├── schema │ │ │ │ │ ├── SQLParserSchemaManager.java │ │ │ │ │ ├── SchemaChangeHelper.java │ │ │ │ │ ├── SchemaChangeManager.java │ │ │ │ │ └── SchemaChangeMode.java │ │ │ │ ├── util │ │ │ │ │ └── DeleteOperation.java │ │ │ │ └── writer │ │ │ │ │ ├── CacheRecordBuffer.java │ │ │ │ │ ├── ChangeEvent.java │ │ │ │ │ ├── DorisAbstractWriter.java │ │ │ │ │ ├── DorisStreamLoad.java │ │ │ │ │ ├── DorisWriteMetrics.java │ │ │ │ │ ├── DorisWriter.java │ │ │ │ │ ├── DorisWriterState.java │ │ │ │ │ ├── DorisWriterStateSerializer.java │ │ │ │ │ ├── EventType.java │ │ │ │ │ ├── LabelGenerator.java │ │ │ │ │ ├── LoadConstants.java │ │ │ │ │ ├── RecordBuffer.java │ │ │ │ │ ├── RecordStream.java │ │ │ │ │ ├── WriteMode.java │ │ │ │ │ └── serializer │ │ │ │ │ ├── DorisRecord.java │ │ │ │ │ ├── DorisRecordSerializer.java │ │ │ │ │ ├── JsonDebeziumSchemaSerializer.java │ │ │ │ │ ├── RecordWithMetaSerializer.java │ │ │ │ │ ├── RowDataSerializer.java │ │ │ │ │ ├── RowSerializer.java │ │ │ │ │ ├── SimpleStringSerializer.java │ │ │ │ │ └── jsondebezium │ │ │ │ │ ├── CdcDataChange.java │ │ │ │ │ ├── CdcSchemaChange.java │ │ │ │ │ ├── JsonDebeziumChangeContext.java │ │ │ │ │ ├── JsonDebeziumChangeUtils.java │ │ │ │ │ ├── JsonDebeziumDataChange.java │ │ │ │ │ ├── JsonDebeziumSchemaChange.java │ │ │ │ │ ├── JsonDebeziumSchemaChangeImpl.java │ │ │ │ │ ├── JsonDebeziumSchemaChangeImplV2.java │ │ │ │ │ └── SQLParserSchemaChange.java │ │ │ │ ├── source │ │ │ │ ├── DorisSource.java │ │ │ │ ├── assigners │ │ │ │ │ ├── DorisSplitAssigner.java │ │ │ │ │ └── SimpleSplitAssigner.java │ │ │ │ ├── enumerator │ │ │ │ │ ├── DorisSourceEnumerator.java │ │ │ │ │ ├── PendingSplitsCheckpoint.java │ │ │ │ │ └── PendingSplitsCheckpointSerializer.java │ │ │ │ ├── reader │ │ │ │ │ ├── DorisFlightValueReader.java │ │ │ │ │ ├── DorisRecordEmitter.java │ │ │ │ │ ├── DorisSourceReader.java │ │ │ │ │ ├── DorisSourceSplitReader.java │ │ │ │ │ ├── DorisValueReader.java │ │ │ │ │ └── ValueReader.java │ │ │ │ └── split │ │ │ │ │ ├── DorisSourceSplit.java │ │ │ │ │ ├── DorisSourceSplitSerializer.java │ │ │ │ │ ├── DorisSourceSplitState.java │ │ │ │ │ └── DorisSplitRecords.java │ │ │ │ ├── table │ │ │ │ ├── DorisConfigOptions.java │ │ │ │ ├── DorisDynamicTableFactory.java │ │ │ │ ├── DorisDynamicTableSink.java │ │ │ │ ├── DorisDynamicTableSource.java │ │ │ │ ├── DorisExpressionVisitor.java │ │ │ │ ├── DorisRowDataAsyncLookupFunction.java │ │ │ │ ├── DorisRowDataInputFormat.java │ │ │ │ ├── DorisRowDataJdbcLookupFunction.java │ │ │ │ └── DorisTableInputSplit.java │ │ │ │ ├── tools │ │ │ │ └── cdc │ │ │ │ │ ├── CdcTools.java │ │ │ │ │ ├── DatabaseSync.java │ │ │ │ │ ├── DatabaseSyncConfig.java │ │ │ │ │ ├── DorisTableConfig.java │ │ │ │ │ ├── JdbcSourceSchema.java │ │ │ │ │ ├── ParsingProcessFunction.java │ │ │ │ │ ├── SourceConnector.java │ │ │ │ │ ├── SourceSchema.java │ │ │ │ │ ├── converter │ │ │ │ │ └── TableNameConverter.java │ │ │ │ │ ├── db2 │ │ │ │ │ ├── Db2DatabaseSync.java │ │ │ │ │ ├── Db2DateConverter.java │ │ │ │ │ ├── Db2Schema.java │ │ │ │ │ └── Db2Type.java │ │ │ │ │ ├── deserialize │ │ │ │ │ └── DorisJsonDebeziumDeserializationSchema.java │ │ │ │ │ ├── mongodb │ │ │ │ │ ├── ChangeStreamConstant.java │ │ │ │ │ ├── MongoDBDatabaseSync.java │ │ │ │ │ ├── MongoDBSchema.java │ │ │ │ │ ├── MongoDBType.java │ │ │ │ │ ├── MongoDateConverter.java │ │ │ │ │ ├── MongoParsingProcessFunction.java │ │ │ │ │ └── serializer │ │ │ │ │ │ ├── MongoDBJsonDebeziumSchemaSerializer.java │ │ │ │ │ │ ├── MongoJsonDebeziumDataChange.java │ │ │ │ │ │ └── MongoJsonDebeziumSchemaChange.java │ │ │ │ │ ├── mysql │ │ │ │ │ ├── DateToStringConverter.java │ │ │ │ │ ├── MysqlDatabaseSync.java │ │ │ │ │ ├── MysqlSchema.java │ │ │ │ │ └── MysqlType.java │ │ │ │ │ ├── oracle │ │ │ │ │ ├── OracleDatabaseSync.java │ │ │ │ │ ├── OracleDateConverter.java │ │ │ │ │ ├── OracleSchema.java │ │ │ │ │ └── OracleType.java │ │ │ │ │ ├── postgres │ │ │ │ │ ├── PostgresDatabaseSync.java │ │ │ │ │ ├── PostgresDateConverter.java │ │ │ │ │ ├── PostgresSchema.java │ │ │ │ │ └── PostgresType.java │ │ │ │ │ ├── sqlserver │ │ │ │ │ ├── SqlServerDatabaseSync.java │ │ │ │ │ ├── SqlServerDateConverter.java │ │ │ │ │ ├── SqlServerSchema.java │ │ │ │ │ └── SqlServerType.java │ │ │ │ │ └── utils │ │ │ │ │ ├── DorisTableUtil.java │ │ │ │ │ └── JsonNodeExtractUtil.java │ │ │ │ └── util │ │ │ │ ├── ErrorMessages.java │ │ │ │ ├── FastDateUtil.java │ │ │ │ └── IPUtils.java │ │ │ └── flink │ │ │ └── table │ │ │ └── runtime │ │ │ └── arrow │ │ │ └── serializers │ │ │ └── ArrowSerializer.java │ └── resources │ │ ├── META-INF │ │ └── services │ │ │ └── org.apache.flink.table.factories.Factory │ │ └── log4j2.properties │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── doris │ │ └── flink │ │ ├── backend │ │ └── BackendClientTest.java │ │ ├── catalog │ │ ├── DorisCatalogFactoryTest.java │ │ ├── DorisCatalogITCase.java │ │ ├── DorisCatalogOptionsTest.java │ │ ├── DorisCatalogTest.java │ │ ├── DorisTypeMapperTest.java │ │ └── doris │ │ │ ├── DorisSchemaFactoryTest.java │ │ │ └── DorisSystemTest.java │ │ ├── cfg │ │ ├── DorisExecutionOptionsTest.java │ │ ├── DorisLookupOptionsTest.java │ │ ├── DorisOptionsTest.java │ │ └── DorisReadOptionsTest.java │ │ ├── connection │ │ └── SimpleJdbcConnectionProviderTest.java │ │ ├── container │ │ ├── AbstractContainerTestBase.java │ │ ├── AbstractE2EService.java │ │ ├── AbstractITCaseService.java │ │ ├── ContainerUtils.java │ │ ├── e2e │ │ │ ├── Doris2DorisE2ECase.java │ │ │ └── Mysql2DorisE2ECase.java │ │ └── instance │ │ │ ├── ContainerService.java │ │ │ ├── DorisContainer.java │ │ │ ├── DorisCustomerContainer.java │ │ │ └── MySQLContainer.java │ │ ├── deserialization │ │ ├── RowDataDeserializationSchemaTest.java │ │ └── convert │ │ │ └── DorisRowConverterTest.java │ │ ├── example │ │ ├── CDCSchemaChangeExample.java │ │ ├── CatalogExample.java │ │ ├── DataGen2DorisExample.java │ │ ├── DorisDateAndTimestampSqlTest.java │ │ ├── DorisIntranetAccessSinkExample.java │ │ ├── DorisSinkArraySQLExample.java │ │ ├── DorisSinkBatchExample.java │ │ ├── DorisSinkExample.java │ │ ├── DorisSinkExampleRowData.java │ │ ├── DorisSinkMultiTableExample.java │ │ ├── DorisSinkSQLExample.java │ │ ├── DorisSinkStreamMultiTableExample.java │ │ ├── DorisSourceDataStream.java │ │ ├── DorisSourceExample.java │ │ ├── DorisSourceSinkExample.java │ │ ├── LookupJoinCdcExample.java │ │ └── LookupJoinExample.java │ │ ├── lookup │ │ ├── DorisLookupTableITCase.java │ │ └── RecordTest.java │ │ ├── rest │ │ ├── SchemaUtilsTest.java │ │ ├── TestPartitionDefinition.java │ │ ├── TestRestService.java │ │ └── models │ │ │ └── TestSchema.java │ │ ├── serialization │ │ ├── TestRouting.java │ │ └── TestRowBatch.java │ │ ├── sink │ │ ├── DorisSinkFailoverITCase.java │ │ ├── DorisSinkITCase.java │ │ ├── DorisSinkMultiTblFailoverITCase.java │ │ ├── DorisSinkTest.java │ │ ├── HttpEntityMock.java │ │ ├── HttpTestUtil.java │ │ ├── OptionUtils.java │ │ ├── TestBackendUtil.java │ │ ├── TestDorisCommittable.java │ │ ├── TestDorisCommittableSerializer.java │ │ ├── TestEscapeHandler.java │ │ ├── TestResponseUtil.java │ │ ├── TestUtil.java │ │ ├── batch │ │ │ ├── TestBatchBufferHttpEntity.java │ │ │ ├── TestBatchBufferStream.java │ │ │ ├── TestDorisBatchSink.java │ │ │ ├── TestDorisBatchStreamLoad.java │ │ │ ├── TestDorisBatchWriter.java │ │ │ └── TestRecordWithMeta.java │ │ ├── committer │ │ │ ├── MockCommitRequest.java │ │ │ └── TestDorisCommitter.java │ │ ├── copy │ │ │ ├── TestBatchRecordBuffer.java │ │ │ ├── TestCopyCommittableSerializer.java │ │ │ ├── TestDorisCopyCommitter.java │ │ │ └── TestDorisCopyWriter.java │ │ ├── schema │ │ │ ├── SQLParserSchemaManagerTest.java │ │ │ ├── SchemaChangeHelperTest.java │ │ │ ├── SchemaManagerITCase.java │ │ │ └── SchemaManagerTest.java │ │ └── writer │ │ │ ├── TestCacheRecordBuffer.java │ │ │ ├── TestDorisStreamLoad.java │ │ │ ├── TestDorisWriter.java │ │ │ ├── TestDorisWriterStateSerializer.java │ │ │ ├── TestJsonDebeziumSchemaSerializer.java │ │ │ ├── TestLabelGenerator.java │ │ │ ├── TestRecordBuffer.java │ │ │ ├── TestRecordWithMetaSerializer.java │ │ │ ├── TestRowDataSerializer.java │ │ │ ├── TestRowSerializer.java │ │ │ └── serializer │ │ │ └── jsondebezium │ │ │ ├── TestJsonDebeziumChangeBase.java │ │ │ ├── TestJsonDebeziumDataChange.java │ │ │ ├── TestJsonDebeziumSchemaChangeImpl.java │ │ │ ├── TestJsonDebeziumSchemaChangeImplV2.java │ │ │ └── TestSQLParserSchemaChange.java │ │ ├── source │ │ ├── DorisSourceExampleTest.java │ │ ├── DorisSourceITCase.java │ │ ├── enumerator │ │ │ ├── DorisSourceEnumeratorTest.java │ │ │ └── PendingSplitsCheckpointSerializerTest.java │ │ ├── reader │ │ │ ├── DorisSourceReaderTest.java │ │ │ └── TestingReaderContext.java │ │ └── split │ │ │ ├── DorisSourceSplitSerializerTest.java │ │ │ ├── DorisSourceSplitTest.java │ │ │ └── DorisSplitRecordsTest.java │ │ ├── table │ │ ├── DorisDynamicTableFactoryTest.java │ │ ├── DorisDynamicTableSourceTest.java │ │ └── DorisRowDataJdbcLookupFunctionITCase.java │ │ ├── tools │ │ └── cdc │ │ │ ├── CdcDb2SyncDatabaseCase.java │ │ │ ├── CdcMongoSyncDatabaseCase.java │ │ │ ├── CdcMysqlSyncDatabaseCase.java │ │ │ ├── CdcOraclelSyncDatabaseCase.java │ │ │ ├── CdcPostgresSyncDatabaseCase.java │ │ │ ├── CdcSqlServerSyncDatabaseCase.java │ │ │ ├── CdcToolsTest.java │ │ │ ├── DatabaseSyncTest.java │ │ │ ├── DorisTableConfigTest.java │ │ │ ├── MockSourceSchema.java │ │ │ ├── db2 │ │ │ └── Db2TypeTest.java │ │ │ └── mongodb │ │ │ ├── MongoDBCreateTableTest.java │ │ │ ├── MongoDBDatabaseSyncTest.java │ │ │ ├── MongoDBSchemaTest.java │ │ │ ├── MongoDBTypeTest.java │ │ │ ├── MongoDateConverterTest.java │ │ │ └── MongoParsingProcessFunctionTest.java │ │ └── utils │ │ ├── DateToStringConverter.java │ │ ├── FactoryMocks.java │ │ ├── FastDateUtilTest.java │ │ ├── MockMultiTableSource.java │ │ └── MockSource.java │ └── resources │ ├── container │ └── e2e │ │ ├── doris2doris │ │ ├── test_doris2doris_sink_test_tbl.sql │ │ └── test_doris2doris_source_test_tbl.sql │ │ └── mysql2doris │ │ ├── testAutoAddTable.txt │ │ ├── testAutoAddTable_init.sql │ │ ├── testMySQL2Doris.txt │ │ ├── testMySQL2DorisByDefault.txt │ │ ├── testMySQL2DorisByDefault_init.sql │ │ ├── testMySQL2DorisCreateTable.txt │ │ ├── testMySQL2DorisCreateTable_init.sql │ │ ├── testMySQL2DorisEnableDelete.txt │ │ ├── testMySQL2DorisEnableDelete_init.sql │ │ ├── testMySQL2DorisMultiDb2One.txt │ │ ├── testMySQL2DorisMultiDb2One_init.sql │ │ ├── testMySQL2DorisMultiDbSync.txt │ │ ├── testMySQL2DorisMultiDbSync_init.sql │ │ ├── testMySQL2DorisSQLParse.txt │ │ ├── testMySQL2DorisSQLParse_init.sql │ │ └── testMySQL2Doris_init.sql │ ├── docker │ └── doris │ │ ├── be.conf │ │ └── fe.conf │ └── log4j2-test.properties ├── mvnw └── tools ├── maven ├── checkstyle.xml └── suppressions.xml └── releasing ├── create_release_branch.sh ├── create_source_release.sh ├── deploy_staging_jars.sh └── update_branch_version.sh /.asf.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | github: 19 | description: Flink Connector for Apache Doris 20 | homepage: https://doris.apache.org/ 21 | labels: 22 | - data-warehousing 23 | - mpp 24 | - olap 25 | - dbms 26 | - apache 27 | - doris 28 | - flink 29 | - connector 30 | enabled_merge_buttons: 31 | squash: true 32 | merge: false 33 | rebase: false 34 | protected_branches: 35 | master: 36 | required_pull_request_reviews: 37 | dismiss_stale_reviews: true 38 | required_approving_review_count: 1 39 | features: 40 | issues: true 41 | projects: true 42 | notifications: 43 | issues: commits@doris.apache.org 44 | commits: commits@doris.apache.org 45 | pullrequests: commits@doris.apache.org 46 | 47 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/config.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | blank_issues_enabled: true 19 | contact_links: 20 | - name: Ask a question or get support 21 | url: https://github.com/apache/incubator-doris/discussions 22 | about: Ask a question or request support for using Apache Doris 23 | 24 | -------------------------------------------------------------------------------- /.github/PULL_REQUEST_TEMPLATE.md: -------------------------------------------------------------------------------- 1 | # Proposed changes 2 | 3 | Issue Number: close #xxx 4 | 5 | ## Problem Summary: 6 | 7 | Describe the overview of changes. 8 | 9 | ## Checklist(Required) 10 | 11 | 1. Does it affect the original behavior: (Yes/No/I Don't know) 12 | 2. Has unit tests been added: (Yes/No/No Need) 13 | 3. Has document been added or modified: (Yes/No/No Need) 14 | 4. Does it need to update dependencies: (Yes/No) 15 | 5. Are there any changes that cannot be rolled back: (Yes/No) 16 | 17 | ## Further comments 18 | 19 | If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc... 20 | -------------------------------------------------------------------------------- /.github/workflows/approve-label-trigger.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | # 18 | --- 19 | name: Label when reviewed 20 | on: pull_request_review 21 | jobs: 22 | 23 | label-when-reviewed: 24 | name: "Label PRs when reviewed" 25 | runs-on: ubuntu-latest 26 | steps: 27 | - name: "Do nothing. Only trigger corresponding workflow_run event" 28 | run: echo 29 | -------------------------------------------------------------------------------- /.github/workflows/checkstyle.yaml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | # 18 | --- 19 | name: Code Style Checker 20 | 21 | on: 22 | pull_request: 23 | 24 | jobs: 25 | java-checkstyle: 26 | name: "CheckStyle" 27 | runs-on: ubuntu-latest 28 | steps: 29 | - name: Checkout 30 | uses: actions/checkout@v3 31 | with: 32 | persist-credentials: false 33 | submodules: recursive 34 | 35 | - name: Setup java 36 | uses: actions/setup-java@v2 37 | with: 38 | distribution: adopt 39 | java-version: '8' 40 | 41 | - name: Run java checkstyle 42 | run: 43 | cd flink-doris-connector && mvn clean compile checkstyle:checkstyle -------------------------------------------------------------------------------- /.github/workflows/license-eyes.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | # 18 | --- 19 | name: License Check 20 | on: 21 | pull_request: 22 | push: 23 | branches: 24 | - master 25 | jobs: 26 | license-check: 27 | name: "License Check" 28 | runs-on: ubuntu-latest 29 | steps: 30 | - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" 31 | uses: actions/checkout@v2 32 | - name: Check License 33 | uses: apache/skywalking-eyes@v0.2.0 34 | env: 35 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 36 | -------------------------------------------------------------------------------- /.github/workflows/run-e2ecase.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | # 18 | --- 19 | name: Run E2ECases 20 | on: 21 | pull_request: 22 | push: 23 | 24 | jobs: 25 | build-extension: 26 | name: "Run E2ECases" 27 | runs-on: ubuntu-latest 28 | defaults: 29 | run: 30 | shell: bash 31 | steps: 32 | - name: Checkout 33 | uses: actions/checkout@master 34 | 35 | - name: Setup java 36 | uses: actions/setup-java@v2 37 | with: 38 | distribution: adopt 39 | java-version: '8' 40 | 41 | - name: Run E2ECases 42 | run: | 43 | cd flink-doris-connector && mvn test -Dtest="*E2ECase" -Dimage="apache/doris:doris-all-in-one-2.1.0" 44 | 45 | -------------------------------------------------------------------------------- /.github/workflows/run-itcase.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | # 18 | --- 19 | name: Run ITCases 20 | on: 21 | pull_request: 22 | push: 23 | 24 | jobs: 25 | build-extension: 26 | name: "Run ITCases" 27 | runs-on: ubuntu-latest 28 | defaults: 29 | run: 30 | shell: bash 31 | steps: 32 | - name: Checkout 33 | uses: actions/checkout@master 34 | 35 | - name: Setup java 36 | uses: actions/setup-java@v2 37 | with: 38 | distribution: adopt 39 | java-version: '8' 40 | 41 | - name: Run ITCases 42 | run: | 43 | cd flink-doris-connector && mvn test -Dtest="*ITCase" -Dimage="apache/doris:doris-all-in-one-2.1.0" 44 | 45 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | custom_env.sh 2 | flink-doris-connector/dependency-reduced-pom.xml 3 | flink-doris-connector/output/ 4 | flink-doris-connector/target/ 5 | flink-doris-connector/pom.xml.releaseBackup 6 | flink-doris-connector/release.properties 7 | .idea 8 | *.iml 9 | *.DS_Store 10 | dist 11 | -------------------------------------------------------------------------------- /.licenserc.yaml: -------------------------------------------------------------------------------- 1 | header: 2 | license: 3 | spdx-id: Apache-2.0 4 | copyright-owner: Apache Software Foundation 5 | 6 | paths-ignore: 7 | - 'dist' 8 | - 'LICENSE.txt' 9 | - 'NOTICE.txt' 10 | - 'NOTICE' 11 | - '.gitignore' 12 | - '.github/PULL_REQUEST_TEMPLATE.md' 13 | - '.licenserc.yaml' 14 | - 'custom_env.sh.tpl' 15 | - 'flink-doris-connector/src/test/resources/container/' 16 | 17 | comment: on-failure 18 | -------------------------------------------------------------------------------- /.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # https://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.4/apache-maven-3.8.4-bin.zip 19 | wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.1/maven-wrapper-3.1.1.jar 20 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | 19 | 20 | # Contribute to Doris and its surrounding projects 21 | 22 | Thank you very much for your interest in the Doris and its surrounding projects. We welcome your suggestions, comments (including criticisms), comments and contributions to the Doris. 23 | 24 | For more information on how to contribute to Doris, please go to the Doris main repository [CONTRIBUTING](https://github.com/apache/doris/blob/master/CONTRIBUTING.md) 25 | -------------------------------------------------------------------------------- /CONTRIBUTING_CN.md: -------------------------------------------------------------------------------- 1 | 19 | 20 | # 为 Doris 及其周边项目做贡献 21 | 22 | 非常感谢您对 Doris 及其周边项目感兴趣,我们非常欢迎您对 Doris 的各种建议、意见(包括批评)、评论和贡献。 23 | 24 | 关于如何为 Doris 及周边项目做贡献,请前往 Doris 主库查阅 [CONTRIBUTING_CN](https://github.com/apache/doris/blob/master/CONTRIBUTING_CN.md) 25 | -------------------------------------------------------------------------------- /NOTICE.txt: -------------------------------------------------------------------------------- 1 | Flink Connector for Apache Doris 2 | Copyright 2018-2025 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /custom_env.sh.tpl: -------------------------------------------------------------------------------- 1 | #export MVN_BIN= 2 | #export JAVA_HOME= 3 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalogOptions.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.catalog; 19 | 20 | import org.apache.flink.configuration.ConfigOption; 21 | import org.apache.flink.configuration.ConfigOptions; 22 | import org.apache.flink.table.catalog.CommonCatalogOptions; 23 | 24 | import java.util.HashMap; 25 | import java.util.Map; 26 | 27 | public class DorisCatalogOptions { 28 | public static final ConfigOption DEFAULT_DATABASE = 29 | ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) 30 | .stringType() 31 | .noDefaultValue(); 32 | 33 | public static final String TABLE_PROPERTIES_PREFIX = "table.properties."; 34 | 35 | public static Map getCreateTableProps(Map tableOptions) { 36 | final Map tableProps = new HashMap<>(); 37 | 38 | for (Map.Entry entry : tableOptions.entrySet()) { 39 | if (entry.getKey().startsWith(TABLE_PROPERTIES_PREFIX)) { 40 | String subKey = entry.getKey().substring(TABLE_PROPERTIES_PREFIX.length()); 41 | tableProps.put(subKey, entry.getValue()); 42 | } 43 | } 44 | return tableProps; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DataModel.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.catalog.doris; 19 | 20 | public enum DataModel { 21 | DUPLICATE, 22 | UNIQUE, 23 | UNIQUE_MOR, 24 | AGGREGATE 25 | } 26 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisType.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.catalog.doris; 19 | 20 | public class DorisType { 21 | public static final String BOOLEAN = "BOOLEAN"; 22 | public static final String TINYINT = "TINYINT"; 23 | public static final String SMALLINT = "SMALLINT"; 24 | public static final String INT = "INT"; 25 | public static final String BIGINT = "BIGINT"; 26 | public static final String LARGEINT = "LARGEINT"; 27 | // largeint is bigint unsigned in information_schema.COLUMNS 28 | public static final String BIGINT_UNSIGNED = "BIGINT UNSIGNED"; 29 | public static final String FLOAT = "FLOAT"; 30 | public static final String DOUBLE = "DOUBLE"; 31 | public static final String DECIMAL = "DECIMAL"; 32 | public static final String DECIMAL_V3 = "DECIMALV3"; 33 | public static final String DATE = "DATE"; 34 | public static final String DATE_V2 = "DATEV2"; 35 | public static final String DATETIME = "DATETIME"; 36 | public static final String DATETIME_V2 = "DATETIMEV2"; 37 | public static final String CHAR = "CHAR"; 38 | public static final String VARCHAR = "VARCHAR"; 39 | public static final String STRING = "STRING"; 40 | public static final String HLL = "HLL"; 41 | public static final String BITMAP = "BITMAP"; 42 | public static final String ARRAY = "ARRAY"; 43 | public static final String JSONB = "JSONB"; 44 | public static final String JSON = "JSON"; 45 | public static final String MAP = "MAP"; 46 | public static final String STRUCT = "STRUCT"; 47 | public static final String VARIANT = "VARIANT"; 48 | public static final String IPV4 = "IPV4"; 49 | public static final String IPV6 = "IPV6"; 50 | } 51 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/connection/JdbcConnectionProvider.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.connection; 19 | 20 | import java.sql.Connection; 21 | 22 | public interface JdbcConnectionProvider { 23 | 24 | /** Get existing connection or establish an new one if there is none. */ 25 | Connection getOrEstablishConnection() throws Exception; 26 | 27 | /** Close possible existing connection. */ 28 | void closeConnection(); 29 | } 30 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.deserialization; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 22 | import org.apache.flink.util.Collector; 23 | 24 | import java.io.Serializable; 25 | import java.util.List; 26 | 27 | /** 28 | * The deserialization schema describes how to turn the doris list record into data types 29 | * (Java/Scala objects) that are processed by Flink. 30 | */ 31 | @PublicEvolving 32 | public interface DorisDeserializationSchema extends Serializable, ResultTypeQueryable { 33 | 34 | void deserialize(List record, Collector out) throws Exception; 35 | } 36 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/RowDataDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.deserialization; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.table.data.GenericRowData; 23 | import org.apache.flink.table.data.RowData; 24 | import org.apache.flink.table.types.logical.RowType; 25 | import org.apache.flink.util.Collector; 26 | 27 | import org.apache.doris.flink.deserialization.converter.DorisRowConverter; 28 | 29 | import java.util.List; 30 | 31 | /** 32 | * A simple implementation of {@link DorisDeserializationSchema} which converts the received list 33 | * record into {@link GenericRowData}. 34 | */ 35 | @PublicEvolving 36 | public class RowDataDeserializationSchema implements DorisDeserializationSchema { 37 | 38 | private final DorisRowConverter rowConverter; 39 | 40 | public RowDataDeserializationSchema(RowType rowType) { 41 | this.rowConverter = new DorisRowConverter(rowType); 42 | } 43 | 44 | @Override 45 | public TypeInformation getProducedType() { 46 | return TypeInformation.of(RowData.class); 47 | } 48 | 49 | @Override 50 | public void deserialize(List record, Collector out) throws Exception { 51 | RowData row = rowConverter.convertInternal(record); 52 | out.collect(row); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/SimpleListDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.deserialization; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.api.common.typeinfo.TypeHint; 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | import org.apache.flink.util.Collector; 24 | 25 | import java.util.List; 26 | 27 | @PublicEvolving 28 | public class SimpleListDeserializationSchema implements DorisDeserializationSchema> { 29 | 30 | @Override 31 | public TypeInformation> getProducedType() { 32 | return TypeInformation.of(new TypeHint>() {}); 33 | } 34 | 35 | @Override 36 | public void deserialize(List record, Collector> out) throws Exception { 37 | out.collect(record); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ConnectedFailedException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class ConnectedFailedException extends DorisRuntimeException { 21 | public ConnectedFailedException(String server, Throwable cause) { 22 | super("Connect to " + server + " failed.", cause); 23 | } 24 | 25 | public ConnectedFailedException(String server, int statusCode, Throwable cause) { 26 | super("Connect to " + server + " failed, status code is " + statusCode + ".", cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/CopyLoadException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class CopyLoadException extends DorisRuntimeException { 21 | public CopyLoadException() { 22 | super(); 23 | } 24 | 25 | public CopyLoadException(String message) { 26 | super(message); 27 | } 28 | 29 | public CopyLoadException(String message, Throwable cause) { 30 | super(message, cause); 31 | } 32 | 33 | public CopyLoadException(Throwable cause) { 34 | super(cause); 35 | } 36 | 37 | protected CopyLoadException( 38 | String message, 39 | Throwable cause, 40 | boolean enableSuppression, 41 | boolean writableStackTrace) { 42 | super(message, cause, enableSuppression, writableStackTrace); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/CreateTableException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | /** Create Table exception. */ 21 | public class CreateTableException extends RuntimeException { 22 | public CreateTableException() { 23 | super(); 24 | } 25 | 26 | public CreateTableException(String message) { 27 | super(message); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisBatchLoadException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | /** Doris batch load run exception. */ 21 | public class DorisBatchLoadException extends RuntimeException { 22 | public DorisBatchLoadException() { 23 | super(); 24 | } 25 | 26 | public DorisBatchLoadException(String message) { 27 | super(message); 28 | } 29 | 30 | public DorisBatchLoadException(String message, Throwable cause) { 31 | super(message, cause); 32 | } 33 | 34 | public DorisBatchLoadException(Throwable cause) { 35 | super(cause); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class DorisException extends Exception { 21 | public DorisException() { 22 | super(); 23 | } 24 | 25 | public DorisException(String message) { 26 | super(message); 27 | } 28 | 29 | public DorisException(String message, Throwable cause) { 30 | super(message, cause); 31 | } 32 | 33 | public DorisException(Throwable cause) { 34 | super(cause); 35 | } 36 | 37 | protected DorisException( 38 | String message, 39 | Throwable cause, 40 | boolean enableSuppression, 41 | boolean writableStackTrace) { 42 | super(message, cause, enableSuppression, writableStackTrace); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisInternalException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | import org.apache.doris.sdk.thrift.TStatusCode; 21 | 22 | import java.util.List; 23 | 24 | public class DorisInternalException extends DorisRuntimeException { 25 | public DorisInternalException(String server, TStatusCode statusCode, List errorMsgs) { 26 | super( 27 | "Doris server " 28 | + server 29 | + " internal failed, status code [" 30 | + statusCode 31 | + "] error message is " 32 | + errorMsgs); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisRuntimeException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | /** Doris runtime exception. */ 21 | public class DorisRuntimeException extends RuntimeException { 22 | public DorisRuntimeException() { 23 | super(); 24 | } 25 | 26 | public DorisRuntimeException(String message) { 27 | super(message); 28 | } 29 | 30 | public DorisRuntimeException(String message, Throwable cause) { 31 | super(message, cause); 32 | } 33 | 34 | public DorisRuntimeException(Throwable cause) { 35 | super(cause); 36 | } 37 | 38 | protected DorisRuntimeException( 39 | String message, 40 | Throwable cause, 41 | boolean enableSuppression, 42 | boolean writableStackTrace) { 43 | super(message, cause, enableSuppression, writableStackTrace); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisSchemaChangeException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | /** Doris Schema Change run exception. */ 21 | public class DorisSchemaChangeException extends RuntimeException { 22 | public DorisSchemaChangeException() { 23 | super(); 24 | } 25 | 26 | public DorisSchemaChangeException(String message) { 27 | super(message); 28 | } 29 | 30 | public DorisSchemaChangeException(String message, Throwable cause) { 31 | super(message, cause); 32 | } 33 | 34 | public DorisSchemaChangeException(Throwable cause) { 35 | super(cause); 36 | } 37 | 38 | protected DorisSchemaChangeException( 39 | String message, 40 | Throwable cause, 41 | boolean enableSuppression, 42 | boolean writableStackTrace) { 43 | super(message, cause, enableSuppression, writableStackTrace); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisSystemException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | /** Doris System run exception. */ 21 | public class DorisSystemException extends RuntimeException { 22 | public DorisSystemException() { 23 | super(); 24 | } 25 | 26 | public DorisSystemException(String message) { 27 | super(message); 28 | } 29 | 30 | public DorisSystemException(String message, Throwable cause) { 31 | super(message, cause); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/IllegalArgumentException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class IllegalArgumentException extends DorisException { 21 | public IllegalArgumentException(String msg, Throwable cause) { 22 | super(msg, cause); 23 | } 24 | 25 | public IllegalArgumentException(String arg, String value) { 26 | super("argument '" + arg + "' is illegal, value is '" + value + "'."); 27 | } 28 | 29 | public IllegalArgumentException(String msg) { 30 | super(msg); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/LabelAlreadyExistsException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class LabelAlreadyExistsException extends DorisRuntimeException { 21 | public LabelAlreadyExistsException() { 22 | super(); 23 | } 24 | 25 | public LabelAlreadyExistsException(String message) { 26 | super(message); 27 | } 28 | 29 | public LabelAlreadyExistsException(String message, Throwable cause) { 30 | super(message, cause); 31 | } 32 | 33 | public LabelAlreadyExistsException(Throwable cause) { 34 | super(cause); 35 | } 36 | 37 | protected LabelAlreadyExistsException( 38 | String message, 39 | Throwable cause, 40 | boolean enableSuppression, 41 | boolean writableStackTrace) { 42 | super(message, cause, enableSuppression, writableStackTrace); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ShouldNeverHappenException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class ShouldNeverHappenException extends DorisRuntimeException {} 21 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.exception; 19 | 20 | public class StreamLoadException extends Exception { 21 | public StreamLoadException() { 22 | super(); 23 | } 24 | 25 | public StreamLoadException(String message) { 26 | super(message); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/lookup/DorisLookupReader.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.lookup; 19 | 20 | import org.apache.flink.table.data.RowData; 21 | 22 | import java.io.Closeable; 23 | import java.io.IOException; 24 | import java.util.List; 25 | import java.util.concurrent.CompletableFuture; 26 | 27 | public abstract class DorisLookupReader implements Closeable { 28 | 29 | public abstract CompletableFuture> asyncGet(RowData record) throws IOException; 30 | 31 | public abstract List get(RowData record) throws IOException; 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/lookup/Get.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.lookup; 19 | 20 | import java.util.List; 21 | import java.util.concurrent.CompletableFuture; 22 | 23 | public class Get { 24 | Record record; 25 | CompletableFuture> future; 26 | 27 | public Get(Record record) { 28 | this.record = record; 29 | } 30 | 31 | public Record getRecord() { 32 | return record; 33 | } 34 | 35 | public CompletableFuture> getFuture() { 36 | return this.future; 37 | } 38 | 39 | public void setFuture(CompletableFuture> future) { 40 | this.future = future; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/lookup/GetAction.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.lookup; 19 | 20 | import java.util.List; 21 | import java.util.concurrent.Semaphore; 22 | 23 | public class GetAction { 24 | 25 | List getList; 26 | 27 | Semaphore semaphore; 28 | 29 | public GetAction(List getList) { 30 | this.getList = getList; 31 | } 32 | 33 | public List getGetList() { 34 | return getList; 35 | } 36 | 37 | public Semaphore getSemaphore() { 38 | return semaphore; 39 | } 40 | 41 | public void setSemaphore(Semaphore semaphore) { 42 | this.semaphore = semaphore; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/lookup/LookupMetrics.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.lookup; 19 | 20 | import org.apache.flink.metrics.Counter; 21 | import org.apache.flink.metrics.MetricGroup; 22 | 23 | import java.io.Serializable; 24 | 25 | /** lookup join metrics. */ 26 | public class LookupMetrics implements Serializable { 27 | public static final String HIT_COUNT = "hitCount"; 28 | public static final String MISS_COUNT = "missCount"; 29 | public static final String LOAD_COUNT = "loadCount"; 30 | private transient Counter hitCounter; 31 | private transient Counter missCounter; 32 | private transient Counter loadCounter; 33 | 34 | public LookupMetrics(MetricGroup metricGroup) { 35 | hitCounter = metricGroup.counter(HIT_COUNT); 36 | missCounter = metricGroup.counter(MISS_COUNT); 37 | loadCounter = metricGroup.counter(LOAD_COUNT); 38 | } 39 | 40 | public void incHitCount() { 41 | hitCounter.inc(); 42 | } 43 | 44 | public void incMissCount() { 45 | missCounter.inc(); 46 | } 47 | 48 | public void incLoadCount() { 49 | loadCounter.inc(); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/lookup/Record.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.lookup; 19 | 20 | import org.apache.flink.table.types.DataType; 21 | 22 | import java.io.Serializable; 23 | import java.util.Arrays; 24 | 25 | /** record. */ 26 | public class Record implements Serializable { 27 | 28 | LookupSchema schema; 29 | Object[] values; 30 | 31 | public Record(LookupSchema schema) { 32 | this.schema = schema; 33 | values = new Object[schema.getFieldTypes().length]; 34 | } 35 | 36 | public LookupSchema getSchema() { 37 | return schema; 38 | } 39 | 40 | public Object getObject(int index) { 41 | return values[index]; 42 | } 43 | 44 | public void setObject(int index, Object obj) { 45 | values[index] = obj; 46 | } 47 | 48 | public String getTableIdentifier() { 49 | return schema.getTableIdentifier(); 50 | } 51 | 52 | public String[] getSelectFields() { 53 | return schema.getSelectFields(); 54 | } 55 | 56 | public String[] getConditionFields() { 57 | return schema.getConditionFields(); 58 | } 59 | 60 | public DataType[] getFieldTypes() { 61 | return schema.getFieldTypes(); 62 | } 63 | 64 | public int[] getKeyIndex() { 65 | return schema.getKeyIndex(); 66 | } 67 | 68 | public Object[] getValues() { 69 | return values; 70 | } 71 | 72 | @Override 73 | public String toString() { 74 | return "Record{" + "schema=" + schema + ", values=" + Arrays.toString(values) + '}'; 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/QueryPlan.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.rest.models; 19 | 20 | import com.fasterxml.jackson.annotation.JsonIgnoreProperties; 21 | import com.fasterxml.jackson.annotation.JsonProperty; 22 | 23 | import java.util.Map; 24 | 25 | @JsonIgnoreProperties(ignoreUnknown = true) 26 | public class QueryPlan { 27 | @JsonProperty(value = "status") 28 | private int status; 29 | 30 | @JsonProperty(value = "opaqued_query_plan") 31 | private String opaquedQueryPlan; 32 | 33 | @JsonProperty(value = "partitions") 34 | private Map partitions; 35 | 36 | public int getStatus() { 37 | return status; 38 | } 39 | 40 | public void setStatus(int status) { 41 | this.status = status; 42 | } 43 | 44 | public String getOpaquedQueryPlan() { 45 | return opaquedQueryPlan; 46 | } 47 | 48 | public void setOpaquedQueryPlan(String opaquedQueryPlan) { 49 | this.opaquedQueryPlan = opaquedQueryPlan; 50 | } 51 | 52 | public Map getPartitions() { 53 | return partitions; 54 | } 55 | 56 | public void setPartitions(Map partitions) { 57 | this.partitions = partitions; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Tablet.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.rest.models; 19 | 20 | import com.fasterxml.jackson.annotation.JsonIgnoreProperties; 21 | 22 | import java.util.List; 23 | 24 | @JsonIgnoreProperties(ignoreUnknown = true) 25 | public class Tablet { 26 | private List routings; 27 | private int version; 28 | private long versionHash; 29 | private long schemaHash; 30 | 31 | public List getRoutings() { 32 | return routings; 33 | } 34 | 35 | public void setRoutings(List routings) { 36 | this.routings = routings; 37 | } 38 | 39 | public int getVersion() { 40 | return version; 41 | } 42 | 43 | public void setVersion(int version) { 44 | this.version = version; 45 | } 46 | 47 | public long getVersionHash() { 48 | return versionHash; 49 | } 50 | 51 | public void setVersionHash(long versionHash) { 52 | this.versionHash = versionHash; 53 | } 54 | 55 | public long getSchemaHash() { 56 | return schemaHash; 57 | } 58 | 59 | public void setSchemaHash(long schemaHash) { 60 | this.schemaHash = schemaHash; 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/Routing.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.serialization; 19 | 20 | import org.apache.doris.flink.exception.IllegalArgumentException; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | import static org.apache.doris.flink.util.ErrorMessages.PARSE_NUMBER_FAILED_MESSAGE; 25 | 26 | /** present an Doris BE address. */ 27 | public class Routing { 28 | private static Logger logger = LoggerFactory.getLogger(Routing.class); 29 | 30 | private String host; 31 | private int port; 32 | 33 | public Routing(String routing) throws IllegalArgumentException { 34 | parseRouting(routing); 35 | } 36 | 37 | private void parseRouting(String routing) throws IllegalArgumentException { 38 | logger.debug("Parse Doris BE address: '{}'.", routing); 39 | String[] hostPort = routing.split(":"); 40 | if (hostPort.length != 2) { 41 | logger.error("Format of Doris BE address '{}' is illegal.", routing); 42 | throw new IllegalArgumentException("Doris BE", routing); 43 | } 44 | this.host = hostPort[0]; 45 | try { 46 | this.port = Integer.parseInt(hostPort[1]); 47 | } catch (NumberFormatException e) { 48 | logger.error(PARSE_NUMBER_FAILED_MESSAGE, "Doris BE's port", hostPort[1]); 49 | throw new IllegalArgumentException("Doris BE", routing); 50 | } 51 | } 52 | 53 | public String getHost() { 54 | return host; 55 | } 56 | 57 | public int getPort() { 58 | return port; 59 | } 60 | 61 | @Override 62 | public String toString() { 63 | return "Doris BE{" + "host='" + host + '\'' + ", port=" + port + '}'; 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/DorisAbstractCommittable.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | public interface DorisAbstractCommittable {} 21 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/DorisCommittableSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.apache.flink.core.io.SimpleVersionedSerializer; 21 | 22 | import java.io.ByteArrayInputStream; 23 | import java.io.ByteArrayOutputStream; 24 | import java.io.DataInputStream; 25 | import java.io.DataOutputStream; 26 | import java.io.IOException; 27 | 28 | /** define how to serialize DorisCommittable. */ 29 | class DorisCommittableSerializer implements SimpleVersionedSerializer { 30 | 31 | @Override 32 | public int getVersion() { 33 | return 1; 34 | } 35 | 36 | @Override 37 | public byte[] serialize(DorisCommittable dorisCommittable) throws IOException { 38 | try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); 39 | final DataOutputStream out = new DataOutputStream(baos)) { 40 | out.writeUTF(dorisCommittable.getHostPort()); 41 | out.writeUTF(dorisCommittable.getDb()); 42 | out.writeLong(dorisCommittable.getTxnID()); 43 | 44 | out.flush(); 45 | return baos.toByteArray(); 46 | } 47 | } 48 | 49 | @Override 50 | public DorisCommittable deserialize(int version, byte[] serialized) throws IOException { 51 | try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized); 52 | final DataInputStream in = new DataInputStream(bais)) { 53 | final String hostPort = in.readUTF(); 54 | final String db = in.readUTF(); 55 | final long txnId = in.readLong(); 56 | return new DorisCommittable(hostPort, db, txnId); 57 | } 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/HttpGetWithEntity.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.apache.http.client.methods.HttpEntityEnclosingRequestBase; 21 | 22 | import java.net.URI; 23 | 24 | public class HttpGetWithEntity extends HttpEntityEnclosingRequestBase { 25 | private static final String METHOD_NAME = "GET"; 26 | 27 | @Override 28 | public String getMethod() { 29 | return METHOD_NAME; 30 | } 31 | 32 | public HttpGetWithEntity(final String uri) { 33 | super(); 34 | setURI(URI.create(uri)); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/LoadStatus.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | /** enum of LoadStatus. */ 21 | public class LoadStatus { 22 | public static final String SUCCESS = "Success"; 23 | public static final String PUBLISH_TIMEOUT = "Publish Timeout"; 24 | public static final String LABEL_ALREADY_EXIST = "Label Already Exists"; 25 | public static final String FAIL = "Fail"; 26 | } 27 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/ResponseUtil.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import java.util.regex.Pattern; 21 | 22 | /** util for handle response. */ 23 | public class ResponseUtil { 24 | public static final Pattern LABEL_EXIST_PATTERN = 25 | Pattern.compile("Label \\[(.*)\\] has already been used, relate to txn \\[(\\d+)\\]"); 26 | public static final Pattern COMMITTED_PATTERN = 27 | Pattern.compile( 28 | "transaction \\[(\\d+)\\] is already \\b(COMMITTED|committed|VISIBLE|visible)\\b, not pre-committed."); 29 | 30 | public static final Pattern ABORTTED_PATTERN = 31 | Pattern.compile( 32 | "transaction \\[(\\d+)\\] is already|transaction \\[(\\d+)\\] not found"); 33 | 34 | public static boolean isCommitted(String msg) { 35 | return COMMITTED_PATTERN.matcher(msg).find(); 36 | } 37 | 38 | public static boolean isAborted(String msg) { 39 | return ABORTTED_PATTERN.matcher(msg).find(); 40 | } 41 | 42 | static final Pattern COPY_COMMITTED_PATTERN = 43 | Pattern.compile("errCode = 2, detailMessage = No files can be copied.*"); 44 | 45 | public static boolean isCopyCommitted(String msg) { 46 | return COPY_COMMITTED_PATTERN.matcher(msg).find(); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/batch/RecordWithMeta.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.batch; 19 | 20 | public class RecordWithMeta { 21 | private String database; 22 | private String table; 23 | private String record; 24 | 25 | public RecordWithMeta() {} 26 | 27 | public RecordWithMeta(String database, String table, String record) { 28 | this.database = database; 29 | this.table = table; 30 | this.record = record; 31 | } 32 | 33 | public String getDatabase() { 34 | return database; 35 | } 36 | 37 | public void setDatabase(String database) { 38 | this.database = database; 39 | } 40 | 41 | public String getTable() { 42 | return table; 43 | } 44 | 45 | public void setTable(String table) { 46 | this.table = table; 47 | } 48 | 49 | public String getRecord() { 50 | return record; 51 | } 52 | 53 | public void setRecord(String record) { 54 | this.record = record; 55 | } 56 | 57 | public String getTableIdentifier() { 58 | return this.database + "." + this.table; 59 | } 60 | 61 | @Override 62 | public String toString() { 63 | return "RecordWithMeta{" 64 | + "database='" 65 | + database 66 | + '\'' 67 | + ", table='" 68 | + table 69 | + '\'' 70 | + ", record='" 71 | + record 72 | + '\'' 73 | + '}'; 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/DorisCopyCommittable.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.copy; 19 | 20 | import org.apache.doris.flink.sink.DorisAbstractCommittable; 21 | 22 | import java.util.Objects; 23 | 24 | public class DorisCopyCommittable implements DorisAbstractCommittable { 25 | private final String hostPort; 26 | private final String copySQL; 27 | 28 | public DorisCopyCommittable(String hostPort, String copySQL) { 29 | this.hostPort = hostPort; 30 | this.copySQL = copySQL; 31 | } 32 | 33 | public String getHostPort() { 34 | return hostPort; 35 | } 36 | 37 | public String getCopySQL() { 38 | return copySQL; 39 | } 40 | 41 | @Override 42 | public boolean equals(Object o) { 43 | if (this == o) { 44 | return true; 45 | } 46 | if (o == null || getClass() != o.getClass()) { 47 | return false; 48 | } 49 | DorisCopyCommittable that = (DorisCopyCommittable) o; 50 | return Objects.equals(hostPort, that.hostPort) && Objects.equals(copySQL, that.copySQL); 51 | } 52 | 53 | @Override 54 | public int hashCode() { 55 | return Objects.hash(hostPort, copySQL); 56 | } 57 | 58 | @Override 59 | public String toString() { 60 | return "DorisCommittable{" 61 | + "hostPort='" 62 | + hostPort 63 | + '\'' 64 | + ", copySQL='" 65 | + copySQL 66 | + '\'' 67 | + '}'; 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/models/BaseResponse.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.copy.models; 19 | 20 | import com.fasterxml.jackson.annotation.JsonIgnoreProperties; 21 | 22 | @JsonIgnoreProperties(ignoreUnknown = true) 23 | public class BaseResponse { 24 | private int code; 25 | private String msg; 26 | private T data; 27 | private int count; 28 | 29 | public int getCode() { 30 | return code; 31 | } 32 | 33 | public String getMsg() { 34 | return msg; 35 | } 36 | 37 | public T getData() { 38 | return data; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/models/CopyIntoResp.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.copy.models; 19 | 20 | import com.fasterxml.jackson.annotation.JsonIgnoreProperties; 21 | 22 | import java.util.Map; 23 | 24 | @JsonIgnoreProperties(ignoreUnknown = true) 25 | public class CopyIntoResp extends BaseResponse { 26 | private String code; 27 | private String exception; 28 | 29 | private Map result; 30 | 31 | public String getDataCode() { 32 | return code; 33 | } 34 | 35 | public String getException() { 36 | return exception; 37 | } 38 | 39 | public Map getResult() { 40 | return result; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/schema/SchemaChangeMode.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.schema; 19 | 20 | public enum SchemaChangeMode { 21 | DEBEZIUM_STRUCTURE("debezium_structure"), 22 | SQL_PARSER("sql_parser"); 23 | 24 | private final String name; 25 | 26 | SchemaChangeMode(String name) { 27 | this.name = name; 28 | } 29 | 30 | public String getName() { 31 | return name; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/util/DeleteOperation.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.util; 19 | 20 | import java.util.Map; 21 | 22 | import static org.apache.doris.flink.sink.writer.LoadConstants.DORIS_DELETE_SIGN; 23 | 24 | public class DeleteOperation { 25 | public static void addDeleteSign(Map valueMap, boolean delete) { 26 | if (delete) { 27 | valueMap.put(DORIS_DELETE_SIGN, "1"); 28 | } else { 29 | valueMap.put(DORIS_DELETE_SIGN, "0"); 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/ChangeEvent.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer; 19 | 20 | import java.io.Serializable; 21 | 22 | /** 23 | * represents the change events of external systems, including data change and schema change event. 24 | */ 25 | public interface ChangeEvent extends Serializable {} 26 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/DorisAbstractWriter.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer; 19 | 20 | import org.apache.flink.api.connector.sink2.StatefulSink; 21 | import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; 22 | 23 | /** Abstract for different Doris Writer: Stream Load, Copy ... */ 24 | public interface DorisAbstractWriter 25 | extends StatefulSink.StatefulSinkWriter, 26 | TwoPhaseCommittingSink.PrecommittingSinkWriter {} 27 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/EventType.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer; 19 | 20 | public enum EventType { 21 | ALTER, 22 | CREATE 23 | } 24 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/LoadConstants.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer; 19 | 20 | import java.util.ArrayList; 21 | import java.util.Arrays; 22 | import java.util.List; 23 | 24 | import static org.apache.doris.flink.sink.LoadStatus.PUBLISH_TIMEOUT; 25 | import static org.apache.doris.flink.sink.LoadStatus.SUCCESS; 26 | 27 | /** Constants for load. */ 28 | public class LoadConstants { 29 | public static final String COLUMNS_KEY = "columns"; 30 | public static final String FIELD_DELIMITER_KEY = "column_separator"; 31 | public static final String FIELD_DELIMITER_DEFAULT = "\t"; 32 | public static final String LINE_DELIMITER_KEY = "line_delimiter"; 33 | public static final String LINE_DELIMITER_DEFAULT = "\n"; 34 | public static final String FORMAT_KEY = "format"; 35 | public static final String JSON = "json"; 36 | public static final String CSV = "csv"; 37 | public static final String ARROW = "arrow"; 38 | public static final String NULL_VALUE = "\\N"; 39 | public static final String DORIS_DELETE_SIGN = "__DORIS_DELETE_SIGN__"; 40 | public static final String READ_JSON_BY_LINE = "read_json_by_line"; 41 | public static final String GROUP_COMMIT = "group_commit"; 42 | public static final String GROUP_COMMIT_OFF_MODE = "off_mode"; 43 | public static final String COMPRESS_TYPE = "compress_type"; 44 | public static final String COMPRESS_TYPE_GZ = "gz"; 45 | public static final List DORIS_SUCCESS_STATUS = 46 | new ArrayList<>(Arrays.asList(SUCCESS, PUBLISH_TIMEOUT)); 47 | } 48 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/WriteMode.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer; 19 | 20 | public enum WriteMode { 21 | STREAM_LOAD, 22 | STREAM_LOAD_BATCH, 23 | COPY; 24 | 25 | public static WriteMode of(String name) { 26 | try { 27 | return WriteMode.valueOf(name.toUpperCase()); 28 | } catch (IllegalArgumentException e) { 29 | throw new IllegalArgumentException("Unsupported write mode: " + name); 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/DorisRecordSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer.serializer; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | 22 | import java.io.IOException; 23 | import java.io.Serializable; 24 | 25 | /** 26 | * How to serialize the record to bytes. 27 | * 28 | * @param 29 | */ 30 | @PublicEvolving 31 | public interface DorisRecordSerializer extends Serializable { 32 | 33 | /** 34 | * define how to convert record into byte array. 35 | * 36 | * @param record 37 | * @return [tableIdentifer,byte array] 38 | * @throws IOException 39 | */ 40 | DorisRecord serialize(T record) throws IOException; 41 | 42 | default void initial() {} 43 | 44 | default DorisRecord flush() { 45 | return DorisRecord.empty; 46 | } 47 | 48 | default void close() throws Exception {} 49 | } 50 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/RecordWithMetaSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer.serializer; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | 22 | import org.apache.commons.lang3.StringUtils; 23 | import org.apache.doris.flink.sink.batch.RecordWithMeta; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import java.io.IOException; 28 | import java.nio.charset.StandardCharsets; 29 | 30 | @PublicEvolving 31 | public class RecordWithMetaSerializer implements DorisRecordSerializer { 32 | private static final Logger LOG = LoggerFactory.getLogger(RecordWithMetaSerializer.class); 33 | 34 | @Override 35 | public DorisRecord serialize(RecordWithMeta record) throws IOException { 36 | if (StringUtils.isBlank(record.getTable()) 37 | || StringUtils.isBlank(record.getDatabase()) 38 | || record.getRecord() == null) { 39 | LOG.warn( 40 | "Record or meta format is incorrect, ignore record db:{}, table:{}, row:{}", 41 | record.getDatabase(), 42 | record.getTable(), 43 | record.getRecord()); 44 | return null; 45 | } 46 | return DorisRecord.of( 47 | record.getDatabase(), 48 | record.getTable(), 49 | record.getRecord().getBytes(StandardCharsets.UTF_8)); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/SimpleStringSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer.serializer; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | 22 | import java.io.IOException; 23 | import java.nio.charset.StandardCharsets; 24 | 25 | /** String Serializer. */ 26 | @PublicEvolving 27 | public class SimpleStringSerializer implements DorisRecordSerializer { 28 | 29 | @Override 30 | public DorisRecord serialize(String record) throws IOException { 31 | return DorisRecord.of(record.getBytes(StandardCharsets.UTF_8)); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/CdcDataChange.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer.serializer.jsondebezium; 19 | 20 | import com.fasterxml.jackson.databind.JsonNode; 21 | import org.apache.doris.flink.sink.writer.ChangeEvent; 22 | import org.apache.doris.flink.sink.writer.serializer.DorisRecord; 23 | 24 | import java.io.IOException; 25 | import java.util.Map; 26 | 27 | /** 28 | * When cdc connector captures data changes from the source database you need to inherit this class 29 | * to complete the synchronized data changes to Doris schema. Supports data messages serialized to 30 | * json 31 | */ 32 | public abstract class CdcDataChange implements ChangeEvent { 33 | 34 | public abstract DorisRecord serialize(String record, JsonNode recordRoot, String op) 35 | throws IOException; 36 | 37 | protected abstract Map extractBeforeRow(JsonNode record); 38 | 39 | protected abstract Map extractAfterRow(JsonNode record); 40 | } 41 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/CdcSchemaChange.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer.serializer.jsondebezium; 19 | 20 | import com.fasterxml.jackson.databind.JsonNode; 21 | import org.apache.doris.flink.sink.writer.ChangeEvent; 22 | 23 | /** 24 | * When cdc connector captures data changes about source database schema changes, you need to 25 | * inherit this class to complete the synchronized changes to Doris schema. Supports data messages 26 | * serialized to json 27 | */ 28 | public abstract class CdcSchemaChange implements ChangeEvent { 29 | 30 | protected abstract String extractDatabase(JsonNode record); 31 | 32 | protected abstract String extractTable(JsonNode record); 33 | 34 | /** Schema change */ 35 | public abstract boolean schemaChange(JsonNode recordRoot); 36 | 37 | protected abstract String getCdcTableIdentifier(JsonNode record); 38 | } 39 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/source/assigners/DorisSplitAssigner.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.assigners; 19 | 20 | import org.apache.doris.flink.source.enumerator.PendingSplitsCheckpoint; 21 | import org.apache.doris.flink.source.split.DorisSourceSplit; 22 | 23 | import javax.annotation.Nullable; 24 | 25 | import java.util.Collection; 26 | import java.util.Optional; 27 | 28 | /** 29 | * The {@code DorisSplitAssigner} is responsible for deciding what split should be processed. It 30 | * determines split processing order. 31 | */ 32 | public interface DorisSplitAssigner { 33 | 34 | /** 35 | * Gets the next split. 36 | * 37 | *

When this method returns an empty {@code Optional}, then the set of splits is assumed to 38 | * be done and the source will finish once the readers finished their current splits. 39 | */ 40 | Optional getNext(@Nullable String hostname); 41 | 42 | /** 43 | * Adds a set of splits to this assigner. This happens for example when some split processing 44 | * failed and the splits need to be re-added, or when new splits got discovered. 45 | */ 46 | void addSplits(Collection splits); 47 | 48 | /** 49 | * Creates a snapshot of the state of this split assigner, to be stored in a checkpoint. 50 | * 51 | * @param checkpointId The ID of the checkpoint for which the snapshot is created. 52 | * @return an object containing the state of the split enumerator. 53 | */ 54 | PendingSplitsCheckpoint snapshotState(long checkpointId); 55 | } 56 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/source/assigners/SimpleSplitAssigner.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.assigners; 19 | 20 | import org.apache.doris.flink.source.enumerator.PendingSplitsCheckpoint; 21 | import org.apache.doris.flink.source.split.DorisSourceSplit; 22 | import org.slf4j.Logger; 23 | import org.slf4j.LoggerFactory; 24 | 25 | import javax.annotation.Nullable; 26 | 27 | import java.util.ArrayList; 28 | import java.util.Collection; 29 | import java.util.Optional; 30 | 31 | /** The {@code SimpleSplitAssigner} hands out splits in a random order. */ 32 | public class SimpleSplitAssigner implements DorisSplitAssigner { 33 | 34 | private static final Logger LOG = LoggerFactory.getLogger(SimpleSplitAssigner.class); 35 | private final ArrayList splits; 36 | 37 | public SimpleSplitAssigner(Collection splits) { 38 | this.splits = new ArrayList<>(splits); 39 | } 40 | 41 | @Override 42 | public Optional getNext(@Nullable String hostname) { 43 | final int size = splits.size(); 44 | return size == 0 ? Optional.empty() : Optional.of(splits.remove(size - 1)); 45 | } 46 | 47 | @Override 48 | public void addSplits(Collection newSplits) { 49 | LOG.info("Adding splits: {}", newSplits); 50 | splits.addAll(newSplits); 51 | } 52 | 53 | @Override 54 | public PendingSplitsCheckpoint snapshotState(long checkpointId) { 55 | LOG.info("Snapshot splits {} for checkpoint {}", splits, checkpointId); 56 | return new PendingSplitsCheckpoint(splits); 57 | } 58 | 59 | @Override 60 | public String toString() { 61 | return "SimpleSplitAssigner " + splits; 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/source/enumerator/PendingSplitsCheckpoint.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.enumerator; 19 | 20 | import org.apache.doris.flink.source.split.DorisSourceSplit; 21 | 22 | import javax.annotation.Nullable; 23 | 24 | import java.util.Collection; 25 | 26 | /** 27 | * A checkpoint of the current state of the containing the currently pending splits that are not yet 28 | * assigned. 29 | */ 30 | public class PendingSplitsCheckpoint { 31 | 32 | /** The splits in the checkpoint. */ 33 | private final Collection splits; 34 | 35 | /** 36 | * The cached byte representation from the last serialization step. This helps to avoid paying 37 | * repeated serialization cost for the same checkpoint object. This field is used by {@link 38 | * PendingSplitsCheckpointSerializer}. 39 | */ 40 | @Nullable byte[] serializedFormCache; 41 | 42 | public PendingSplitsCheckpoint(Collection splits) { 43 | this.splits = splits; 44 | } 45 | 46 | public Collection getSplits() { 47 | return splits; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/source/reader/DorisRecordEmitter.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.reader; 19 | 20 | import org.apache.flink.api.connector.source.SourceOutput; 21 | import org.apache.flink.connector.base.source.reader.RecordEmitter; 22 | import org.apache.flink.util.Collector; 23 | 24 | import org.apache.doris.flink.deserialization.DorisDeserializationSchema; 25 | import org.apache.doris.flink.source.split.DorisSourceSplitState; 26 | 27 | import java.util.List; 28 | 29 | /** The {@link RecordEmitter} implementation for {@link DorisSourceReader}. */ 30 | public class DorisRecordEmitter implements RecordEmitter { 31 | 32 | private final DorisDeserializationSchema dorisDeserializationSchema; 33 | private final OutputCollector outputCollector; 34 | 35 | public DorisRecordEmitter(DorisDeserializationSchema dorisDeserializationSchema) { 36 | this.dorisDeserializationSchema = dorisDeserializationSchema; 37 | this.outputCollector = new OutputCollector<>(); 38 | } 39 | 40 | @Override 41 | public void emitRecord(List value, SourceOutput output, DorisSourceSplitState splitState) 42 | throws Exception { 43 | outputCollector.output = output; 44 | dorisDeserializationSchema.deserialize(value, outputCollector); 45 | } 46 | 47 | private static class OutputCollector implements Collector { 48 | private SourceOutput output; 49 | 50 | @Override 51 | public void collect(T record) { 52 | output.collect(record); 53 | } 54 | 55 | @Override 56 | public void close() { 57 | // do nothing 58 | } 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/source/split/DorisSourceSplitState.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.split; 19 | 20 | /** State of the reader, essentially a mutable version of the {@link DorisSourceSplit}. */ 21 | public class DorisSourceSplitState { 22 | 23 | private final DorisSourceSplit split; 24 | 25 | public DorisSourceSplitState(DorisSourceSplit split) { 26 | this.split = split; 27 | } 28 | 29 | public DorisSourceSplit toDorisSourceSplit() { 30 | return split; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisTableInputSplit.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.table; 19 | 20 | import org.apache.flink.core.io.InputSplit; 21 | 22 | import org.apache.doris.flink.rest.PartitionDefinition; 23 | 24 | /** DorisTableInputSplit. */ 25 | public class DorisTableInputSplit implements InputSplit, java.io.Serializable { 26 | 27 | /** The number of the split. */ 28 | private final int splitNumber; 29 | 30 | protected final PartitionDefinition partition; 31 | 32 | public DorisTableInputSplit(int splitNumber, PartitionDefinition partition) { 33 | super(); 34 | this.splitNumber = splitNumber; 35 | this.partition = partition; 36 | } 37 | 38 | @Override 39 | public int getSplitNumber() { 40 | return splitNumber; 41 | } 42 | 43 | @Override 44 | public String toString() { 45 | return String.format( 46 | "DorisTableInputSplit: %s.%s,id=%s,be=%s,tablets=%s", 47 | partition.getDatabase(), 48 | partition.getTable(), 49 | splitNumber, 50 | partition.getBeAddress(), 51 | partition.getTabletIds()); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/SourceConnector.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc; 19 | 20 | public enum SourceConnector { 21 | MYSQL("mysql"), 22 | ORACLE("oracle"), 23 | POSTGRES("postgres"), 24 | SQLSERVER("sqlserver"), 25 | MONGODB("mongodb"), 26 | DB2("db2"); 27 | 28 | public final String connectorName; 29 | 30 | SourceConnector(String connectorName) { 31 | this.connectorName = connectorName; 32 | } 33 | 34 | public String getConnectorName() { 35 | return connectorName; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/db2/Db2Schema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.db2; 19 | 20 | import org.apache.doris.flink.catalog.doris.FieldSchema; 21 | import org.apache.doris.flink.tools.cdc.JdbcSourceSchema; 22 | 23 | import java.sql.DatabaseMetaData; 24 | import java.sql.SQLException; 25 | import java.util.LinkedHashMap; 26 | 27 | public class Db2Schema extends JdbcSourceSchema { 28 | public Db2Schema( 29 | DatabaseMetaData metaData, 30 | String databaseName, 31 | String schemaName, 32 | String tableName, 33 | String tableComment) 34 | throws Exception { 35 | super(metaData, databaseName, schemaName, tableName, tableComment); 36 | } 37 | 38 | @Override 39 | public String convertToDorisType(String fieldType, Integer precision, Integer scale) { 40 | return Db2Type.toDorisType(fieldType, precision, scale); 41 | } 42 | 43 | @Override 44 | public String getCdcTableName() { 45 | return schemaName + "\\." + tableName; 46 | } 47 | 48 | @Override 49 | public LinkedHashMap getColumnInfo( 50 | DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) 51 | throws SQLException { 52 | return super.getColumnInfo(metaData, null, schemaName, tableName); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/ChangeStreamConstant.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mongodb; 19 | 20 | import java.io.Serializable; 21 | 22 | public class ChangeStreamConstant implements Serializable { 23 | private static final long serialVersionUID = 2599456667907755804L; 24 | public static final String ID_FIELD = "_id"; 25 | public static final String OID_FIELD = "$oid"; 26 | public static final String FIELD_TYPE = "operationType"; 27 | public static final String FIELD_DATA = "fullDocument"; 28 | public static final String OP_UPDATE = "update"; 29 | public static final String OP_INSERT = "insert"; 30 | public static final String OP_REPLACE = "replace"; 31 | public static final String OP_DELETE = "delete"; 32 | public static final String FIELD_DATABASE = "db"; 33 | public static final String FIELD_TABLE = "coll"; 34 | public static final String FIELD_NAMESPACE = "ns"; 35 | public static final String FIELD_DOCUMENT_KEY = "documentKey"; 36 | 37 | public static final String DATE_FIELD = "$date"; 38 | public static final String TIMESTAMP_FIELD = "$timestamp"; 39 | 40 | public static final String DECIMAL_FIELD = "$numberDecimal"; 41 | 42 | public static final String LONG_FIELD = "$numberLong"; 43 | } 44 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDateConverter.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mongodb; 19 | 20 | import org.apache.doris.flink.tools.cdc.DatabaseSyncConfig; 21 | 22 | import java.time.Instant; 23 | import java.time.LocalDateTime; 24 | import java.time.ZoneId; 25 | import java.time.format.DateTimeFormatter; 26 | 27 | public class MongoDateConverter { 28 | private static final ThreadLocal dateFormatterThreadLocal = 29 | ThreadLocal.withInitial( 30 | () -> DateTimeFormatter.ofPattern(DatabaseSyncConfig.DATETIME_MICRO_FORMAT)); 31 | 32 | public static String convertTimestampToString(long timestamp) { 33 | Instant instant = Instant.ofEpochMilli(timestamp); 34 | LocalDateTime localDateTime = 35 | LocalDateTime.ofInstant(instant, ZoneId.of(DatabaseSyncConfig.TIME_ZONE_SHANGHAI)); 36 | return dateFormatterThreadLocal.get().format(localDateTime); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoParsingProcessFunction.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mongodb; 19 | 20 | import com.fasterxml.jackson.databind.JsonNode; 21 | import com.fasterxml.jackson.databind.node.NullNode; 22 | import org.apache.doris.flink.tools.cdc.ParsingProcessFunction; 23 | import org.apache.doris.flink.tools.cdc.converter.TableNameConverter; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | public class MongoParsingProcessFunction extends ParsingProcessFunction { 28 | private static final Logger LOG = LoggerFactory.getLogger(MongoParsingProcessFunction.class); 29 | 30 | public MongoParsingProcessFunction(String databaseName, TableNameConverter converter) { 31 | super(databaseName, converter); 32 | } 33 | 34 | @Override 35 | protected String getRecordTableName(String record) throws Exception { 36 | JsonNode jsonNode = objectMapper.readValue(record, JsonNode.class); 37 | if (jsonNode.get("ns") == null || jsonNode.get("ns") instanceof NullNode) { 38 | LOG.error("Failed to get cdc namespace"); 39 | throw new RuntimeException(); 40 | } 41 | JsonNode nameSpace = jsonNode.get("ns"); 42 | return extractJsonNode(nameSpace, "coll"); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mysql/MysqlSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mysql; 19 | 20 | import org.apache.doris.flink.tools.cdc.JdbcSourceSchema; 21 | 22 | import java.sql.DatabaseMetaData; 23 | 24 | public class MysqlSchema extends JdbcSourceSchema { 25 | 26 | public MysqlSchema( 27 | DatabaseMetaData metaData, String databaseName, String tableName, String tableComment) 28 | throws Exception { 29 | super(metaData, databaseName, null, tableName, tableComment); 30 | } 31 | 32 | public String convertToDorisType(String fieldType, Integer precision, Integer scale) { 33 | return MysqlType.toDorisType(fieldType, precision, scale); 34 | } 35 | 36 | @Override 37 | public String getCdcTableName() { 38 | return databaseName + "\\." + tableName; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/postgres/PostgresSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.postgres; 19 | 20 | import org.apache.doris.flink.tools.cdc.JdbcSourceSchema; 21 | 22 | import java.sql.DatabaseMetaData; 23 | 24 | public class PostgresSchema extends JdbcSourceSchema { 25 | 26 | public PostgresSchema( 27 | DatabaseMetaData metaData, 28 | String databaseName, 29 | String schemaName, 30 | String tableName, 31 | String tableComment) 32 | throws Exception { 33 | super(metaData, databaseName, schemaName, tableName, tableComment); 34 | } 35 | 36 | @Override 37 | public String convertToDorisType(String fieldType, Integer precision, Integer scale) { 38 | return PostgresType.toDorisType(fieldType, precision, scale); 39 | } 40 | 41 | @Override 42 | public String getCdcTableName() { 43 | return schemaName + "\\." + tableName; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/sqlserver/SqlServerSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.sqlserver; 19 | 20 | import org.apache.doris.flink.tools.cdc.JdbcSourceSchema; 21 | 22 | import java.sql.DatabaseMetaData; 23 | 24 | public class SqlServerSchema extends JdbcSourceSchema { 25 | 26 | public SqlServerSchema( 27 | DatabaseMetaData metaData, 28 | String databaseName, 29 | String schemaName, 30 | String tableName, 31 | String tableComment) 32 | throws Exception { 33 | super(metaData, databaseName, schemaName, tableName, tableComment); 34 | } 35 | 36 | @Override 37 | public String convertToDorisType(String fieldType, Integer precision, Integer scale) { 38 | return SqlServerType.toDorisType(fieldType, precision, scale); 39 | } 40 | 41 | @Override 42 | public String getCdcTableName() { 43 | return schemaName + "\\." + tableName; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/java/org/apache/doris/flink/util/ErrorMessages.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.util; 19 | 20 | public abstract class ErrorMessages { 21 | public static final String PARSE_NUMBER_FAILED_MESSAGE = 22 | "Parse '{}' to number failed. Original string is '{}'."; 23 | public static final String PARSE_BOOL_FAILED_MESSAGE = 24 | "Parse '{}' to boolean failed. Original string is '{}'."; 25 | public static final String CONNECT_FAILED_MESSAGE = "Connect to doris {} failed."; 26 | public static final String ILLEGAL_ARGUMENT_MESSAGE = 27 | "argument '{}' is illegal, value is '{}'."; 28 | public static final String SHOULD_NOT_HAPPEN_MESSAGE = "Should not come here."; 29 | public static final String DORIS_INTERNAL_FAIL_MESSAGE = 30 | "Doris server '{}' internal failed, status is '{}', error message is '{}'"; 31 | } 32 | -------------------------------------------------------------------------------- /flink-doris-connector/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | org.apache.doris.flink.table.DorisDynamicTableFactory 19 | org.apache.doris.flink.catalog.DorisCatalogFactory -------------------------------------------------------------------------------- /flink-doris-connector/src/main/resources/log4j2.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | rootLogger.level = INFO 20 | rootLogger.appenderRef.console.ref = ConsoleAppender 21 | 22 | appender.console.name = ConsoleAppender 23 | appender.console.type = CONSOLE 24 | appender.console.layout.type = PatternLayout 25 | appender.console.layout.pattern = %d{HH:mm:ss,SSS} %-5p %-60c [%t] %x - %m%n 26 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/backend/BackendClientTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.backend; 19 | 20 | import org.apache.doris.flink.cfg.DorisReadOptions; 21 | import org.apache.doris.flink.exception.ConnectedFailedException; 22 | import org.apache.doris.flink.exception.IllegalArgumentException; 23 | import org.apache.doris.flink.serialization.Routing; 24 | import org.junit.Test; 25 | 26 | public class BackendClientTest { 27 | 28 | @Test(expected = ConnectedFailedException.class) 29 | public void testBackendClient() throws IllegalArgumentException { 30 | new BackendClient(new Routing("127.0.0.1:1"), DorisReadOptions.builder().build()); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.catalog; 19 | 20 | import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; 21 | import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; 22 | import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; 23 | import org.apache.flink.table.catalog.exceptions.TableNotExistException; 24 | 25 | import org.apache.doris.flink.cfg.DorisConnectionOptions; 26 | import org.junit.Before; 27 | import org.junit.Test; 28 | 29 | import java.util.HashMap; 30 | import java.util.Map; 31 | 32 | public class DorisCatalogTest { 33 | 34 | private DorisCatalog catalog; 35 | 36 | @Before 37 | public void setup() 38 | throws DatabaseAlreadyExistException, TableAlreadyExistException, 39 | TableNotExistException, DatabaseNotExistException { 40 | DorisConnectionOptions connectionOptions = 41 | new DorisConnectionOptions.DorisConnectionOptionsBuilder() 42 | .withFenodes("127.0.0.1:8030") 43 | .withJdbcUrl("jdbc:mysql://127.0.0.1:8030") 44 | .withUsername("root") 45 | .withPassword("xxxxx") 46 | .build(); 47 | 48 | Map props = new HashMap<>(); 49 | catalog = new DorisCatalog("catalog_test", connectionOptions, "test", props); 50 | } 51 | 52 | @Test(expected = Exception.class) 53 | public void testQueryFenodes() { 54 | catalog.queryFenodes(); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/container/instance/ContainerService.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.container.instance; 19 | 20 | import org.apache.doris.flink.exception.DorisRuntimeException; 21 | 22 | import java.sql.Connection; 23 | 24 | public interface ContainerService { 25 | 26 | void startContainer(); 27 | 28 | default void restartContainer() { 29 | throw new DorisRuntimeException("Only doris docker container can implemented."); 30 | } 31 | 32 | boolean isRunning(); 33 | 34 | Connection getQueryConnection(); 35 | 36 | String getJdbcUrl(); 37 | 38 | String getInstanceHost(); 39 | 40 | Integer getMappedPort(int originalPort); 41 | 42 | String getUsername(); 43 | 44 | String getPassword(); 45 | 46 | default String getFenodes() { 47 | throw new DorisRuntimeException("Only doris container can implemented."); 48 | } 49 | 50 | default String getBenodes() { 51 | throw new DorisRuntimeException("Only doris container can implemented."); 52 | } 53 | 54 | void close(); 55 | } 56 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/example/CatalogExample.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.example; 19 | 20 | import org.apache.flink.api.common.RuntimeExecutionMode; 21 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 22 | import org.apache.flink.table.api.Table; 23 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; 24 | import org.apache.flink.types.Row; 25 | 26 | public class CatalogExample { 27 | 28 | public static void main(String[] args) throws Exception { 29 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 30 | env.setParallelism(1); 31 | env.setRuntimeMode(RuntimeExecutionMode.BATCH); 32 | final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); 33 | tEnv.executeSql( 34 | "CREATE CATALOG doris_catalog WITH(\n" 35 | + "'type' = 'doris',\n" 36 | + "'default-database' = 'test',\n" 37 | + "'username' = 'root',\n" 38 | + "'password' = '',\n" 39 | + "'fenodes' = '1127.0.0.1:8030',\n" 40 | + "'jdbc-url' = 'jdbc:mysql://127.0.0.1:9030',\n" 41 | + "'sink.label-prefix' = 'label'\n" 42 | + ")"); 43 | // define a dynamic aggregating query 44 | final Table result = tEnv.sqlQuery("SELECT * from doris_catalog.test.type_test"); 45 | 46 | // print the result to the console 47 | tEnv.toRetractStream(result, Row.class).print(); 48 | env.execute(); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/rest/models/TestSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.rest.models; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Rule; 22 | import org.junit.Test; 23 | import org.junit.rules.ExpectedException; 24 | 25 | public class TestSchema { 26 | @Rule public ExpectedException thrown = ExpectedException.none(); 27 | 28 | @Test 29 | public void testPutGet() { 30 | Schema ts = new Schema(1); 31 | Field f = new Field(); 32 | ts.put(f); 33 | Assert.assertEquals(f, ts.get(0)); 34 | 35 | thrown.expect(IndexOutOfBoundsException.class); 36 | thrown.expectMessage("Index: 1, Fields size:1"); 37 | ts.get(1); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRouting.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.serialization; 19 | 20 | import org.apache.doris.flink.exception.IllegalArgumentException; 21 | import org.junit.Assert; 22 | import org.junit.Rule; 23 | import org.junit.Test; 24 | import org.junit.rules.ExpectedException; 25 | 26 | import static org.hamcrest.core.StringStartsWith.startsWith; 27 | 28 | public class TestRouting { 29 | @Rule public ExpectedException thrown = ExpectedException.none(); 30 | 31 | @Test 32 | public void testRouting() throws Exception { 33 | Routing r1 = new Routing("10.11.12.13:1234"); 34 | Assert.assertEquals("10.11.12.13", r1.getHost()); 35 | Assert.assertEquals(1234, r1.getPort()); 36 | 37 | thrown.expect(IllegalArgumentException.class); 38 | thrown.expectMessage(startsWith("argument ")); 39 | new Routing("10.11.12.13:wxyz"); 40 | } 41 | 42 | @Test(expected = IllegalArgumentException.class) 43 | public void testRoutingErr() throws IllegalArgumentException { 44 | new Routing("10.11.12.13"); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpEntityMock.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.apache.http.Header; 21 | import org.apache.http.HttpEntity; 22 | import org.apache.http.message.BasicHeader; 23 | 24 | import java.io.ByteArrayInputStream; 25 | import java.io.IOException; 26 | import java.io.InputStream; 27 | import java.io.OutputStream; 28 | import java.nio.charset.StandardCharsets; 29 | 30 | /** Mock Class for HttpEntity. */ 31 | public class HttpEntityMock implements HttpEntity { 32 | private String value; 33 | 34 | public HttpEntityMock() {} 35 | 36 | public void setValue(String value) { 37 | this.value = value; 38 | } 39 | 40 | @Override 41 | public boolean isRepeatable() { 42 | return false; 43 | } 44 | 45 | @Override 46 | public boolean isChunked() { 47 | return false; 48 | } 49 | 50 | @Override 51 | public long getContentLength() { 52 | return 0; 53 | } 54 | 55 | @Override 56 | public Header getContentType() { 57 | return new BasicHeader("header", "text/html;charset=utf-8;"); 58 | } 59 | 60 | @Override 61 | public Header getContentEncoding() { 62 | return null; 63 | } 64 | 65 | @Override 66 | public InputStream getContent() throws IOException, UnsupportedOperationException { 67 | return new ByteArrayInputStream(value.getBytes(StandardCharsets.UTF_8)); 68 | } 69 | 70 | @Override 71 | public void writeTo(OutputStream outputStream) throws IOException {} 72 | 73 | @Override 74 | public boolean isStreaming() { 75 | return false; 76 | } 77 | 78 | @Override 79 | public void consumeContent() throws IOException {} 80 | } 81 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/TestDorisCommittable.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Test; 22 | 23 | public class TestDorisCommittable { 24 | 25 | @Test 26 | public void testDorisCommittableEqual() { 27 | DorisCommittable dorisCommittable = new DorisCommittable("127.0.0.1:8710", "test", 0); 28 | Assert.assertNotEquals(dorisCommittable, null); 29 | 30 | DorisCommittable committable2 = new DorisCommittable("127.0.0.1:8710", "test", 0); 31 | Assert.assertEquals(dorisCommittable, committable2); 32 | Assert.assertEquals(dorisCommittable, dorisCommittable); 33 | Assert.assertNotEquals(dorisCommittable, null); 34 | 35 | committable2 = new DorisCommittable("127.0.0.1:8710", "test", 1); 36 | Assert.assertNotEquals(dorisCommittable, committable2); 37 | 38 | committable2 = new DorisCommittable("127.0.0.1:8710", "test1", 0); 39 | Assert.assertNotEquals(dorisCommittable, committable2); 40 | 41 | committable2 = new DorisCommittable("127.0.0.2:8710", "test", 0); 42 | Assert.assertNotEquals(dorisCommittable, committable2); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/TestDorisCommittableSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Test; 22 | 23 | /** test for DorisCommittableSerializer. */ 24 | public class TestDorisCommittableSerializer { 25 | @Test 26 | public void testSerialize() throws Exception { 27 | DorisCommittable expectCommittable = new DorisCommittable("fe:8040", "test", 100); 28 | DorisCommittableSerializer serializer = new DorisCommittableSerializer(); 29 | DorisCommittable committable = 30 | serializer.deserialize(1, serializer.serialize(expectCommittable)); 31 | Assert.assertEquals(expectCommittable, committable); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/TestEscapeHandler.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Test; 22 | 23 | import java.util.Properties; 24 | 25 | import static org.apache.doris.flink.sink.writer.LoadConstants.FIELD_DELIMITER_KEY; 26 | import static org.apache.doris.flink.sink.writer.LoadConstants.LINE_DELIMITER_KEY; 27 | 28 | /** test for EscapeHandler. */ 29 | public class TestEscapeHandler { 30 | @Test 31 | public void testHandle() { 32 | Properties properties = new Properties(); 33 | properties.setProperty(FIELD_DELIMITER_KEY, "\\x09\\x09"); 34 | properties.setProperty(LINE_DELIMITER_KEY, "\\x0A\\x0A"); 35 | EscapeHandler.handleEscape(properties); 36 | Assert.assertEquals("\t\t", properties.getProperty(FIELD_DELIMITER_KEY)); 37 | Assert.assertEquals("\n\n", properties.getProperty(LINE_DELIMITER_KEY)); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/TestUtil.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink; 19 | 20 | import org.apache.flink.api.common.time.Deadline; 21 | import org.apache.flink.util.function.SupplierWithException; 22 | 23 | import java.util.concurrent.TimeoutException; 24 | 25 | public class TestUtil { 26 | 27 | public static void waitUntilCondition( 28 | SupplierWithException condition, 29 | Deadline timeout, 30 | long retryIntervalMillis, 31 | String errorMsg) 32 | throws Exception { 33 | while (timeout.hasTimeLeft() && !(Boolean) condition.get()) { 34 | long timeLeft = Math.max(0L, timeout.timeLeft().toMillis()); 35 | Thread.sleep(Math.min(retryIntervalMillis, timeLeft)); 36 | } 37 | 38 | if (!timeout.hasTimeLeft()) { 39 | throw new TimeoutException(errorMsg); 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestBatchBufferHttpEntity.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.batch; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Test; 22 | 23 | import java.io.ByteArrayOutputStream; 24 | 25 | import static org.junit.Assert.assertArrayEquals; 26 | import static org.junit.Assert.assertEquals; 27 | import static org.junit.Assert.assertFalse; 28 | import static org.junit.Assert.assertTrue; 29 | 30 | public class TestBatchBufferHttpEntity { 31 | 32 | @Test 33 | public void testWrite() throws Exception { 34 | BatchRecordBuffer recordBuffer = TestBatchBufferStream.mockBuffer(); 35 | byte[] expectedData = TestBatchBufferStream.mergeByteArrays(recordBuffer.getBuffer()); 36 | Assert.assertEquals(recordBuffer.getNumOfRecords(), 1000); 37 | 38 | BatchBufferHttpEntity entity = new BatchBufferHttpEntity(recordBuffer); 39 | assertTrue(entity.isRepeatable()); 40 | assertFalse(entity.isStreaming()); 41 | assertEquals(entity.getContentLength(), expectedData.length); 42 | ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); 43 | entity.writeTo(outputStream); 44 | assertArrayEquals(expectedData, outputStream.toByteArray()); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestDorisBatchSink.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.batch; 19 | 20 | import org.apache.doris.flink.cfg.DorisExecutionOptions; 21 | import org.apache.doris.flink.cfg.DorisOptions; 22 | import org.apache.doris.flink.cfg.DorisReadOptions; 23 | import org.apache.doris.flink.sink.writer.serializer.SimpleStringSerializer; 24 | import org.junit.Assert; 25 | import org.junit.Test; 26 | 27 | public class TestDorisBatchSink { 28 | 29 | @Test 30 | public void testBuild() { 31 | DorisBatchSink.Builder builder = DorisBatchSink.builder(); 32 | DorisOptions.Builder dorisBuilder = DorisOptions.builder(); 33 | dorisBuilder 34 | .setFenodes("127.0.0.1:8030") 35 | .setTableIdentifier("db.tbl") 36 | .setUsername("root") 37 | .setPassword(""); 38 | DorisExecutionOptions.Builder executionBuilder = DorisExecutionOptions.builder(); 39 | DorisBatchSink build = 40 | builder.setDorisExecutionOptions(executionBuilder.build()) 41 | .setSerializer(new SimpleStringSerializer()) 42 | .setDorisOptions(dorisBuilder.build()) 43 | .build(); 44 | 45 | DorisReadOptions expected = DorisReadOptions.builder().build(); 46 | DorisReadOptions actual = build.getDorisReadOptions(); 47 | Assert.assertEquals(expected, actual); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestRecordWithMeta.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.batch; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Test; 22 | 23 | public class TestRecordWithMeta { 24 | 25 | @Test 26 | public void testRecordWithMeta() { 27 | RecordWithMeta recordWithMeta = new RecordWithMeta("db", "tbl", "doris,1"); 28 | Assert.assertEquals(recordWithMeta.getRecord(), "doris,1"); 29 | Assert.assertEquals(recordWithMeta.getDatabase(), "db"); 30 | Assert.assertEquals(recordWithMeta.getTable(), "tbl"); 31 | Assert.assertEquals(recordWithMeta.getTableIdentifier(), "db.tbl"); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/committer/MockCommitRequest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.committer; 19 | 20 | import org.apache.flink.api.connector.sink2.Committer; 21 | 22 | public class MockCommitRequest implements Committer.CommitRequest { 23 | 24 | private final CommT committable; 25 | 26 | public MockCommitRequest(CommT committable) { 27 | this.committable = committable; 28 | } 29 | 30 | @Override 31 | public CommT getCommittable() { 32 | return committable; 33 | } 34 | 35 | @Override 36 | public int getNumberOfRetries() { 37 | return 0; 38 | } 39 | 40 | @Override 41 | public void signalFailedWithKnownReason(Throwable throwable) {} 42 | 43 | @Override 44 | public void signalFailedWithUnknownReason(Throwable throwable) {} 45 | 46 | @Override 47 | public void retryLater() {} 48 | 49 | @Override 50 | public void updateAndRetryLater(CommT commT) {} 51 | 52 | @Override 53 | public void signalAlreadyCommitted() {} 54 | } 55 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestCopyCommittableSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.copy; 19 | 20 | import org.junit.Assert; 21 | import org.junit.Test; 22 | 23 | public class TestCopyCommittableSerializer { 24 | 25 | @Test 26 | public void testSerialize() throws Exception { 27 | DorisCopyCommittable expectCommittable = 28 | new DorisCopyCommittable( 29 | "fe:8040", 30 | "COPY INTO db.table FROM @u FILES=('label_0_1') FILE_FORMAT=('type'='csv','line_delimiter'='\n','column_separator'=',')"); 31 | CopyCommittableSerializer serializer = new CopyCommittableSerializer(); 32 | DorisCopyCommittable committable = 33 | serializer.deserialize(1, serializer.serialize(expectCommittable)); 34 | Assert.assertEquals(expectCommittable, committable); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/TestRecordWithMetaSerializer.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.sink.writer; 19 | 20 | import org.apache.doris.flink.sink.batch.RecordWithMeta; 21 | import org.apache.doris.flink.sink.writer.serializer.DorisRecord; 22 | import org.apache.doris.flink.sink.writer.serializer.RecordWithMetaSerializer; 23 | import org.junit.Assert; 24 | import org.junit.Test; 25 | 26 | import java.io.IOException; 27 | 28 | public class TestRecordWithMetaSerializer { 29 | 30 | @Test 31 | public void testSerialize() throws IOException { 32 | RecordWithMetaSerializer serializer = new RecordWithMetaSerializer(); 33 | RecordWithMeta record = new RecordWithMeta(); 34 | record.setRecord("doris,1"); 35 | DorisRecord serialize = serializer.serialize(record); 36 | Assert.assertNull(null, serialize); 37 | 38 | record.setDatabase("database"); 39 | record.setTable("table"); 40 | serialize = serializer.serialize(record); 41 | DorisRecord expected = DorisRecord.of("database", "table", "doris,1".getBytes()); 42 | Assert.assertEquals(expected.getDatabase(), serialize.getDatabase()); 43 | Assert.assertEquals(expected.getTable(), serialize.getTable()); 44 | Assert.assertEquals(new String(expected.getRow()), new String(serialize.getRow())); 45 | Assert.assertEquals(expected.getTableIdentifier(), serialize.getTableIdentifier()); 46 | 47 | Assert.assertNull(serializer.serialize(new RecordWithMeta(null, "table", "doris,1"))); 48 | Assert.assertNull(serializer.serialize(new RecordWithMeta("database", "table", null))); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceExampleTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source; 19 | 20 | import org.apache.flink.api.common.eventtime.WatermarkStrategy; 21 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 22 | import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; 23 | 24 | import org.apache.doris.flink.deserialization.SimpleListDeserializationSchema; 25 | import org.apache.doris.flink.sink.OptionUtils; 26 | import org.junit.Ignore; 27 | import org.junit.Test; 28 | 29 | import java.util.List; 30 | 31 | /** Example Tests for the {@link DorisSource}. */ 32 | @Ignore 33 | public class DorisSourceExampleTest { 34 | 35 | @Test 36 | public void testBoundedDorisSource() throws Exception { 37 | DorisSource> dorisSource = 38 | DorisSource.>builder() 39 | .setDorisOptions(OptionUtils.buildDorisOptions()) 40 | .setDorisReadOptions(OptionUtils.buildDorisReadOptions()) 41 | .setDeserializer(new SimpleListDeserializationSchema()) 42 | .build(); 43 | 44 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 45 | env.setParallelism(1); 46 | env.fromSource(dorisSource, WatermarkStrategy.noWatermarks(), "doris Source") 47 | .addSink(new PrintSinkFunction<>()); 48 | env.execute("Flink doris source test"); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/source/enumerator/PendingSplitsCheckpointSerializerTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.enumerator; 19 | 20 | import org.apache.doris.flink.sink.OptionUtils; 21 | import org.apache.doris.flink.source.split.DorisSourceSplit; 22 | import org.apache.doris.flink.source.split.DorisSourceSplitSerializer; 23 | import org.junit.Assert; 24 | import org.junit.Test; 25 | 26 | import java.util.Arrays; 27 | 28 | /** Unit tests for the {@link PendingSplitsCheckpointSerializer}. */ 29 | public class PendingSplitsCheckpointSerializerTest { 30 | 31 | private static void assertCheckpointsEqual( 32 | final PendingSplitsCheckpoint expected, final PendingSplitsCheckpoint actual) { 33 | Assert.assertEquals(expected.getSplits(), actual.getSplits()); 34 | } 35 | 36 | @Test 37 | public void serializeSplit() throws Exception { 38 | final DorisSourceSplit split = 39 | new DorisSourceSplit("splitId", OptionUtils.buildPartitionDef()); 40 | PendingSplitsCheckpoint checkpoint = new PendingSplitsCheckpoint(Arrays.asList(split)); 41 | 42 | final PendingSplitsCheckpointSerializer splitSerializer = 43 | new PendingSplitsCheckpointSerializer(DorisSourceSplitSerializer.INSTANCE); 44 | byte[] serialized = splitSerializer.serialize(checkpoint); 45 | PendingSplitsCheckpoint deserialize = 46 | splitSerializer.deserialize(splitSerializer.getVersion(), serialized); 47 | 48 | assertCheckpointsEqual(checkpoint, deserialize); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/source/split/DorisSourceSplitSerializerTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.split; 19 | 20 | import org.apache.doris.flink.sink.OptionUtils; 21 | import org.junit.Test; 22 | 23 | import static org.junit.Assert.assertEquals; 24 | 25 | /** Unit tests for the {@link DorisSourceSplitSerializer}. */ 26 | public class DorisSourceSplitSerializerTest { 27 | 28 | @Test 29 | public void serializeSplit() throws Exception { 30 | final DorisSourceSplit split = 31 | new DorisSourceSplit("splitId", OptionUtils.buildPartitionDef()); 32 | 33 | DorisSourceSplit deSerialized = serializeAndDeserializeSplit(split); 34 | assertEquals(split, deSerialized); 35 | } 36 | 37 | private DorisSourceSplit serializeAndDeserializeSplit(DorisSourceSplit split) throws Exception { 38 | final DorisSourceSplitSerializer splitSerializer = new DorisSourceSplitSerializer(); 39 | byte[] serialized = splitSerializer.serialize(split); 40 | return splitSerializer.deserialize(splitSerializer.getVersion(), serialized); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/source/split/DorisSourceSplitTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.split; 19 | 20 | import org.apache.doris.flink.rest.PartitionDefinition; 21 | import org.junit.Assert; 22 | import org.junit.Test; 23 | 24 | import java.util.HashSet; 25 | 26 | public class DorisSourceSplitTest { 27 | 28 | @Test 29 | public void testSplit() { 30 | PartitionDefinition pd1 = 31 | new PartitionDefinition("db", "tbl", "be", new HashSet<>(), "queryplan1"); 32 | PartitionDefinition pd2 = 33 | new PartitionDefinition("db", "tbl", "be", new HashSet<>(), "queryplan1"); 34 | DorisSourceSplit split1 = new DorisSourceSplit("be_1", pd1); 35 | DorisSourceSplit split2 = new DorisSourceSplit("be_2", pd2); 36 | Assert.assertEquals(split1, split2); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/source/split/DorisSplitRecordsTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.source.split; 19 | 20 | import org.junit.Test; 21 | 22 | import java.util.Collections; 23 | 24 | import static org.junit.Assert.assertEquals; 25 | 26 | /** Unit tests for the {@link DorisSplitRecords} class. */ 27 | public class DorisSplitRecordsTest { 28 | 29 | @Test 30 | public void testEmptySplits() { 31 | final String split = "empty"; 32 | final DorisSplitRecords records = DorisSplitRecords.finishedSplit(split); 33 | 34 | assertEquals(Collections.singleton(split), records.finishedSplits()); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/DorisTableConfigTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc; 19 | 20 | import org.apache.flink.api.java.tuple.Tuple2; 21 | 22 | import org.junit.Before; 23 | import org.junit.Test; 24 | 25 | import java.util.Map; 26 | 27 | import static org.junit.Assert.assertEquals; 28 | 29 | public class DorisTableConfigTest { 30 | 31 | private DorisTableConfig dorisTableConfig; 32 | 33 | @Before 34 | public void init() { 35 | dorisTableConfig = new DorisTableConfig(); 36 | } 37 | 38 | @Test 39 | public void buildTableBucketMapTest() { 40 | String tableBuckets = "tbl1:10,tbl2 : 20, a.* :30,b.*:40,.*:50"; 41 | Map tableBucketsMap = dorisTableConfig.buildTableBucketMap(tableBuckets); 42 | assertEquals(10, tableBucketsMap.get("tbl1").intValue()); 43 | assertEquals(20, tableBucketsMap.get("tbl2").intValue()); 44 | assertEquals(30, tableBucketsMap.get("a.*").intValue()); 45 | assertEquals(40, tableBucketsMap.get("b.*").intValue()); 46 | assertEquals(50, tableBucketsMap.get(".*").intValue()); 47 | } 48 | 49 | @Test 50 | public void buildTablePartitionMapTest() { 51 | String tablePartitions = "tbl1:dt_col_d:day,tbl2:dt_col_w:week,tbl3:dt_col_m:month"; 52 | Map> tablePartitionMap = 53 | dorisTableConfig.buildTablePartitionMap(tablePartitions); 54 | assertEquals(Tuple2.of("dt_col_d", "day"), tablePartitionMap.get("tbl1")); 55 | assertEquals(Tuple2.of("dt_col_w", "week"), tablePartitionMap.get("tbl2")); 56 | assertEquals(Tuple2.of("dt_col_m", "month"), tablePartitionMap.get("tbl3")); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/MockSourceSchema.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc; 19 | 20 | public class MockSourceSchema extends SourceSchema { 21 | 22 | public MockSourceSchema(String databaseName, String schemaName, String tableName) 23 | throws Exception { 24 | super(databaseName, schemaName, tableName, ""); 25 | } 26 | 27 | @Override 28 | public String convertToDorisType(String fieldType, Integer precision, Integer scale) { 29 | return null; 30 | } 31 | 32 | @Override 33 | public String getCdcTableName() { 34 | return databaseName + "\\." + tableName; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSyncTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mongodb; 19 | 20 | import org.junit.Before; 21 | import org.junit.Test; 22 | 23 | import java.sql.SQLException; 24 | 25 | import static org.junit.Assert.assertEquals; 26 | 27 | public class MongoDBDatabaseSyncTest { 28 | private MongoDBDatabaseSync mongoDBDatabaseSync; 29 | 30 | @Before 31 | public void init() throws SQLException { 32 | mongoDBDatabaseSync = new MongoDBDatabaseSync(); 33 | } 34 | 35 | @Test 36 | public void testCalculateSampleSize() { 37 | long sampleSize1 = mongoDBDatabaseSync.calculateSampleSize(100L, 0.2, 1000L, 100000L); 38 | long sampleSize2 = mongoDBDatabaseSync.calculateSampleSize(1000L, 0.2, 1000L, 100000L); 39 | long sampleSize3 = mongoDBDatabaseSync.calculateSampleSize(2000L, 0.2, 1000L, 100000L); 40 | long sampleSize4 = mongoDBDatabaseSync.calculateSampleSize(10000L, 0.2, 1000L, 100000L); 41 | long sampleSize5 = mongoDBDatabaseSync.calculateSampleSize(1000000L, 0.2, 1000L, 100000L); 42 | assertEquals(100, sampleSize1); 43 | assertEquals(1000, sampleSize2); 44 | assertEquals(1000, sampleSize3); 45 | assertEquals(2000, sampleSize4); 46 | assertEquals(100000, sampleSize5); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDateConverterTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mongodb; 19 | 20 | import org.junit.Test; 21 | 22 | import static org.junit.Assert.assertEquals; 23 | 24 | public class MongoDateConverterTest { 25 | 26 | @Test 27 | public void convertTimestampToString() { 28 | Long timestamp = 1717488217456L; 29 | String formatStr = MongoDateConverter.convertTimestampToString(timestamp); 30 | assertEquals("2024-06-04 16:03:37.456000", formatStr); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/mongodb/MongoParsingProcessFunctionTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.tools.cdc.mongodb; 19 | 20 | import org.junit.Test; 21 | 22 | import static org.junit.Assert.assertEquals; 23 | 24 | public class MongoParsingProcessFunctionTest { 25 | 26 | @Test 27 | public void getRecordTableName() throws Exception { 28 | String record = 29 | "{\"_id\":\"{\\\"_id\\\": {\\\"$oid\\\": \\\"66583533791a67a6f8c5a339\\\"}}\",\"operationType\":\"insert\",\"fullDocument\":\"{\\\"_id\\\": {\\\"$oid\\\": \\\"66583533791a67a6f8c5a339\\\"}, \\\"key1\\\": \\\"value1\\\"}\",\"source\":{\"ts_ms\":0,\"snapshot\":\"true\"},\"ts_ms\":1717065582062,\"ns\":{\"db\":\"test\",\"coll\":\"cdc_test\"},\"to\":null,\"documentKey\":\"{\\\"_id\\\": {\\\"$oid\\\": \\\"66583533791a67a6f8c5a339\\\"}}\",\"updateDescription\":null,\"clusterTime\":null,\"txnNumber\":null,\"lsid\":null}"; 30 | MongoParsingProcessFunction mongoParsingProcessFunction = 31 | new MongoParsingProcessFunction(null, null); 32 | String recordTableName = mongoParsingProcessFunction.getRecordTableName(record); 33 | assertEquals("cdc_test", recordTableName); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/utils/FactoryMocks.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.utils; 19 | 20 | import org.apache.flink.table.api.DataTypes; 21 | import org.apache.flink.table.catalog.Column; 22 | import org.apache.flink.table.catalog.ObjectIdentifier; 23 | import org.apache.flink.table.catalog.ResolvedSchema; 24 | import org.apache.flink.table.factories.FactoryUtil; 25 | import org.apache.flink.table.types.DataType; 26 | import org.apache.flink.table.types.logical.RowType; 27 | 28 | /** Utilities for testing instances usually created by {@link FactoryUtil}. */ 29 | public final class FactoryMocks { 30 | 31 | public static final ResolvedSchema SCHEMA = 32 | ResolvedSchema.of( 33 | Column.physical("a", DataTypes.STRING()), 34 | Column.physical("b", DataTypes.INT()), 35 | Column.physical("c", DataTypes.BOOLEAN())); 36 | 37 | public static final ResolvedSchema SCHEMA_DT = 38 | ResolvedSchema.of( 39 | Column.physical("a", DataTypes.STRING()), 40 | Column.physical("b", DataTypes.INT()), 41 | Column.physical("c", DataTypes.BOOLEAN()), 42 | Column.physical("d", DataTypes.DATE()), 43 | Column.physical("e", DataTypes.TIMESTAMP())); 44 | 45 | public static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); 46 | 47 | public static final RowType PHYSICAL_TYPE = (RowType) PHYSICAL_DATA_TYPE.getLogicalType(); 48 | 49 | public static final ObjectIdentifier IDENTIFIER = 50 | ObjectIdentifier.of("default", "default", "t1"); 51 | 52 | private FactoryMocks() { 53 | // no instantiation 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/java/org/apache/doris/flink/utils/FastDateUtilTest.java: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | package org.apache.doris.flink.utils; 19 | 20 | import org.apache.doris.flink.util.FastDateUtil; 21 | import org.junit.jupiter.api.Test; 22 | 23 | import java.time.LocalDate; 24 | import java.time.LocalDateTime; 25 | 26 | import static org.junit.jupiter.api.Assertions.assertEquals; 27 | 28 | public class FastDateUtilTest { 29 | 30 | @Test 31 | void fastParseDateTimeV2_withValidDateTimeAndPattern_returnsCorrectLocalDateTime() { 32 | String dateTime = "2023-10-05 14:30:45.123456"; 33 | String pattern = "yyyy-MM-dd HH:mm:ss.SSSSSS"; 34 | LocalDateTime result = FastDateUtil.fastParseDateTimeV2(dateTime, pattern); 35 | assertEquals(LocalDateTime.of(2023, 10, 5, 14, 30, 45, 123456000), result); 36 | } 37 | 38 | @Test 39 | void fastParseDateTime_withValidDateTimeAndPattern_returnsCorrectLocalDateTime() { 40 | String dateTime = "2023-10-05 14:30:45"; 41 | String pattern = "yyyy-MM-dd HH:mm:ss"; 42 | LocalDateTime result = FastDateUtil.fastParseDateTime(dateTime, pattern); 43 | assertEquals(LocalDateTime.of(2023, 10, 5, 14, 30, 45), result); 44 | } 45 | 46 | @Test 47 | void fastParseDate_withValidDateAndPattern_returnsCorrectLocalDate() { 48 | String dateTime = "2023-10-05"; 49 | String pattern = "yyyy-MM-dd"; 50 | LocalDate result = FastDateUtil.fastParseDate(dateTime, pattern); 51 | assertEquals(LocalDate.of(2023, 10, 5), result); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/doris2doris/test_doris2doris_sink_test_tbl.sql: -------------------------------------------------------------------------------- 1 | CREATE DATABASE IF NOT EXISTS test_doris2doris_sink; 2 | 3 | DROP TABLE IF EXISTS test_doris2doris_sink.test_tbl; 4 | 5 | CREATE TABLE test_doris2doris_sink.test_tbl ( 6 | `id` int, 7 | `c1` boolean, 8 | `c2` tinyint, 9 | `c3` smallint, 10 | `c4` int, 11 | `c5` bigint, 12 | `c6` largeint, 13 | `c7` float, 14 | `c8` double, 15 | `c9` decimal(12,4), 16 | `c10` date, 17 | `c11` datetime, 18 | `c12` char(1), 19 | `c13` varchar(256), 20 | `c14` string, 21 | `c15` Array, 22 | `c16` Map, 23 | `c17` Struct, 24 | `c18` JSON, 25 | `c19` VARIANT 26 | ) 27 | DUPLICATE KEY(`id`) 28 | DISTRIBUTED BY HASH(`id`) BUCKETS 1 29 | PROPERTIES ( 30 | "replication_num" = "1", 31 | "light_schema_change" = "true" 32 | ); 33 | 34 | -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/doris2doris/test_doris2doris_source_test_tbl.sql: -------------------------------------------------------------------------------- 1 | CREATE DATABASE IF NOT EXISTS test_doris2doris_source; 2 | 3 | DROP TABLE IF EXISTS test_doris2doris_source.test_tbl; 4 | 5 | CREATE TABLE test_doris2doris_source.test_tbl ( 6 | `id` int, 7 | `c1` boolean, 8 | `c2` tinyint, 9 | `c3` smallint, 10 | `c4` int, 11 | `c5` bigint, 12 | `c6` largeint, 13 | `c7` float, 14 | `c8` double, 15 | `c9` decimal(12,4), 16 | `c10` date, 17 | `c11` datetime, 18 | `c12` char(1), 19 | `c13` varchar(16), 20 | `c14` string, 21 | `c15` Array, 22 | `c16` Map, 23 | `c17` Struct, 24 | `c18` JSON, 25 | `c19` JSON -- doris2.1.0 can not read VARIANT 26 | ) 27 | DUPLICATE KEY(`id`) 28 | DISTRIBUTED BY HASH(`id`) BUCKETS 1 29 | PROPERTIES ( 30 | "replication_num" = "1", 31 | "light_schema_change" = "true" 32 | ); 33 | 34 | INSERT INTO test_doris2doris_source.test_tbl VALUES 35 | (1, true, 127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727, 36 | 3.14, 2.71828, 12345.6789, '2025-03-11', '2025-03-11 12:34:56', 'A', 'Hello, Doris!', 'This is a string', 37 | ['Alice', 'Bob'], {'key1': 'value1', 'key2': 'value2'}, STRUCT('Tom', 30), '{"key": "value"}', '{"type": "variant", "data": 123}'); 38 | 39 | INSERT INTO test_doris2doris_source.test_tbl VALUES 40 | (2, false, -128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728, 41 | -1.23, 0.0001, -9999.9999, '2024-12-25', '2024-12-25 23:59:59', 'B', 'Doris Test', 'Another string!', 42 | ['Charlie', 'David'], {'k1': 'v1', 'k2': 'v2'}, STRUCT('Jerry', 25), '{"status": "ok"}', '{"data": [1, 2, 3]}' ); 43 | 44 | INSERT INTO test_doris2doris_source.test_tbl VALUES 45 | (3, true, 0, 0, 0, 0, 0, 46 | 0.0, 0.0, 0.0000, '2023-06-15', '2023-06-15 08:00:00', 'C', 'Test Doris', 'Sample text', 47 | ['Eve', 'Frank'], {'alpha': 'beta'}, STRUCT('Alice', 40), '{"nested": {"key": "value"}}', '{"variant": "test"}'); 48 | 49 | INSERT INTO test_doris2doris_source.test_tbl VALUES 50 | (4, NULL, NULL, NULL, NULL, NULL, NULL, 51 | NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, 52 | NULL, NULL, NULL, NULL, NULL); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testAutoAddTable.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql 4 | --including-tables "tbl.*|auto_add" 5 | --table-conf replication_num=1 6 | --single-sink true 7 | --ignore-default-value true -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testAutoAddTable_init.sql: -------------------------------------------------------------------------------- 1 | DROP DATABASE if EXISTS test_e2e_mysql; 2 | CREATE DATABASE if NOT EXISTS test_e2e_mysql; 3 | DROP TABLE IF EXISTS test_e2e_mysql.tbl1; 4 | CREATE TABLE test_e2e_mysql.tbl1 ( 5 | `name` varchar(256) primary key, 6 | `age` int 7 | ); 8 | insert into test_e2e_mysql.tbl1 values ('doris_1',1); 9 | 10 | 11 | DROP TABLE IF EXISTS test_e2e_mysql.tbl2; 12 | CREATE TABLE test_e2e_mysql.tbl2 ( 13 | `name` varchar(256) primary key, 14 | `age` int 15 | ); 16 | insert into test_e2e_mysql.tbl2 values ('doris_2',2); 17 | 18 | 19 | DROP TABLE IF EXISTS test_e2e_mysql.tbl3; 20 | CREATE TABLE test_e2e_mysql.tbl3 ( 21 | `name` varchar(256) primary key, 22 | `age` int 23 | ); 24 | insert into test_e2e_mysql.tbl3 values ('doris_3',3); 25 | 26 | 27 | DROP TABLE IF EXISTS test_e2e_mysql.tbl4; 28 | CREATE TABLE test_e2e_mysql.tbl4 ( 29 | `name` varchar(256) primary key, 30 | `age` int 31 | ); 32 | 33 | 34 | DROP TABLE IF EXISTS test_e2e_mysql.tbl5; 35 | CREATE TABLE test_e2e_mysql.tbl5 ( 36 | `name` varchar(256) primary key, 37 | `age` int 38 | ); 39 | insert into test_e2e_mysql.tbl5 values ('doris_5',5); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2Doris.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql 4 | --table-prefix ods_ 5 | --table-suffix _incr 6 | --including-tables "tbl.*" 7 | --excluding-tables "tbl4" 8 | --sink-conf sink.ignore.update-before=false 9 | --table-conf replication_num=1 10 | --single-sink true 11 | --ignore-default-value false 12 | --schema-change-mode sql_parser -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisByDefault.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql 4 | --including-tables "tbl1|tbl2|tbl3|tbl5" 5 | --table-conf replication_num=1 -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisByDefault_init.sql: -------------------------------------------------------------------------------- 1 | DROP DATABASE if EXISTS test_e2e_mysql; 2 | CREATE DATABASE if NOT EXISTS test_e2e_mysql; 3 | DROP TABLE IF EXISTS test_e2e_mysql.tbl1; 4 | CREATE TABLE test_e2e_mysql.tbl1 ( 5 | `name` varchar(256) primary key, 6 | `age` int 7 | ); 8 | insert into test_e2e_mysql.tbl1 values ('doris_1',1); 9 | 10 | 11 | DROP TABLE IF EXISTS test_e2e_mysql.tbl2; 12 | CREATE TABLE test_e2e_mysql.tbl2 ( 13 | `name` varchar(256) primary key, 14 | `age` int 15 | ); 16 | insert into test_e2e_mysql.tbl2 values ('doris_2',2); 17 | 18 | 19 | DROP TABLE IF EXISTS test_e2e_mysql.tbl3; 20 | CREATE TABLE test_e2e_mysql.tbl3 ( 21 | `name` varchar(256) primary key, 22 | `age` int 23 | ); 24 | insert into test_e2e_mysql.tbl3 values ('doris_3',3); 25 | 26 | 27 | DROP TABLE IF EXISTS test_e2e_mysql.tbl4; 28 | CREATE TABLE test_e2e_mysql.tbl4 ( 29 | `name` varchar(256) primary key, 30 | `age` int 31 | ); 32 | 33 | 34 | DROP TABLE IF EXISTS test_e2e_mysql.tbl5; 35 | CREATE TABLE test_e2e_mysql.tbl5 ( 36 | `name` varchar(256) primary key, 37 | `age` int 38 | ); 39 | insert into test_e2e_mysql.tbl5 values ('doris_5',5); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisCreateTable.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql 4 | --including-tables "create_tbl_.*" 5 | --create-table-only 6 | --table-conf table-buckets=create_tbl_uniq:10,create_tbl_from_uniqindex.*:30 7 | --table-conf replication_num=1 8 | --table-conf convert-uniq-to-pk=true -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisCreateTable_init.sql: -------------------------------------------------------------------------------- 1 | DROP DATABASE if EXISTS test_e2e_mysql; 2 | CREATE DATABASE if NOT EXISTS test_e2e_mysql; 3 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_uniq; 4 | CREATE TABLE test_e2e_mysql.create_tbl_uniq ( 5 | `id` int NOT NULL, 6 | `name` varchar(255) DEFAULT NULL, 7 | `age` bigint DEFAULT NULL, 8 | PRIMARY KEY (`id`) USING BTREE 9 | ); 10 | 11 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_dup; 12 | CREATE TABLE test_e2e_mysql.create_tbl_dup ( 13 | `id` int NOT NULL, 14 | `name` varchar(255) DEFAULT NULL, 15 | `age` bigint DEFAULT NULL 16 | ); 17 | 18 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_from_uniqindex; 19 | CREATE TABLE test_e2e_mysql.create_tbl_from_uniqindex ( 20 | `id` int NOT NULL, 21 | `name` varchar(255) DEFAULT NULL, 22 | `age` bigint DEFAULT NULL, 23 | UNIQUE KEY `uniq` (`name`) 24 | ); 25 | 26 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_from_uniqindex2; 27 | CREATE TABLE test_e2e_mysql.create_tbl_from_uniqindex2 ( 28 | `id` int DEFAULT NULL, 29 | `name` varchar(255) DEFAULT NULL, 30 | `age` int DEFAULT NULL, 31 | UNIQUE KEY `idname_uniq` (`id`,`name`), 32 | UNIQUE KEY `idage_uniq` (`id`,`age`) 33 | ); 34 | 35 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_from_multiindex; 36 | CREATE TABLE test_e2e_mysql.create_tbl_from_multiindex ( 37 | `id` int DEFAULT NULL, 38 | `name` varchar(255) DEFAULT NULL, 39 | `age` int DEFAULT NULL, 40 | UNIQUE KEY `uniq` (`id`), 41 | KEY `normal` (`name`) 42 | ); 43 | 44 | -- for auto partition table 45 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_part_uniq; 46 | CREATE TABLE test_e2e_mysql.create_tbl_part_uniq ( 47 | `id` int NOT NULL, 48 | `name` varchar(255) DEFAULT NULL, 49 | `age` int DEFAULT NULL, 50 | `create_dtime` datetime DEFAULT NULL, 51 | PRIMARY KEY (`id`) USING BTREE 52 | ); 53 | 54 | DROP TABLE IF EXISTS test_e2e_mysql.create_tbl_part_dup; 55 | CREATE TABLE test_e2e_mysql.create_tbl_part_dup ( 56 | `id` int NOT NULL, 57 | `name` varchar(255) DEFAULT NULL, 58 | `age` int DEFAULT NULL, 59 | `create_dtime` datetime DEFAULT NULL 60 | ); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisEnableDelete.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql 4 | --including-tables "tbl1|tbl2|tbl3|tbl5" 5 | --table-conf replication_num=1 6 | --sink-conf sink.enable-delete=false 7 | --sink-conf sink.check-interval=5000 -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisEnableDelete_init.sql: -------------------------------------------------------------------------------- 1 | DROP DATABASE if EXISTS test_e2e_mysql; 2 | CREATE DATABASE if NOT EXISTS test_e2e_mysql; 3 | DROP TABLE IF EXISTS test_e2e_mysql.tbl1; 4 | CREATE TABLE test_e2e_mysql.tbl1 ( 5 | `name` varchar(256) primary key, 6 | `age` int 7 | ); 8 | insert into test_e2e_mysql.tbl1 values ('doris_1',1); 9 | 10 | 11 | DROP TABLE IF EXISTS test_e2e_mysql.tbl2; 12 | CREATE TABLE test_e2e_mysql.tbl2 ( 13 | `name` varchar(256) primary key, 14 | `age` int 15 | ); 16 | insert into test_e2e_mysql.tbl2 values ('doris_2',2); 17 | 18 | 19 | DROP TABLE IF EXISTS test_e2e_mysql.tbl3; 20 | CREATE TABLE test_e2e_mysql.tbl3 ( 21 | `name` varchar(256) primary key, 22 | `age` int 23 | ); 24 | insert into test_e2e_mysql.tbl3 values ('doris_3',3); 25 | 26 | 27 | DROP TABLE IF EXISTS test_e2e_mysql.tbl4; 28 | CREATE TABLE test_e2e_mysql.tbl4 ( 29 | `name` varchar(256) primary key, 30 | `age` int 31 | ); 32 | 33 | 34 | DROP TABLE IF EXISTS test_e2e_mysql.tbl5; 35 | CREATE TABLE test_e2e_mysql.tbl5 ( 36 | `name` varchar(256) primary key, 37 | `age` int 38 | ); 39 | insert into test_e2e_mysql.tbl5 values ('doris_5',5); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisMultiDb2One.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql_db.* 4 | --including-tables ".*" 5 | --multi-to-one-origin "tbl2.*" 6 | --multi-to-one-target "tbl2_merge" 7 | --table-conf replication_num=1 -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisMultiDb2One_init.sql: -------------------------------------------------------------------------------- 1 | -- tbl1 2 | DROP DATABASE if EXISTS test_e2e_mysql_db1; 3 | CREATE DATABASE if NOT EXISTS test_e2e_mysql_db1; 4 | DROP TABLE IF EXISTS test_e2e_mysql_db1.tbl1; 5 | CREATE TABLE test_e2e_mysql_db1.tbl1 ( 6 | `id` int NOT NULL, 7 | `name` varchar(255) DEFAULT NULL, 8 | `age` bigint DEFAULT NULL, 9 | PRIMARY KEY (`id`) USING BTREE 10 | ); 11 | insert into test_e2e_mysql_db1.tbl1 values (1,'db1_tb1',18); 12 | 13 | DROP DATABASE if EXISTS test_e2e_mysql_db2; 14 | CREATE DATABASE if NOT EXISTS test_e2e_mysql_db2; 15 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl1; 16 | CREATE TABLE test_e2e_mysql_db2.tbl1 ( 17 | `id` int NOT NULL, 18 | `name` varchar(255) DEFAULT NULL, 19 | `age` bigint DEFAULT NULL, 20 | PRIMARY KEY (`id`) USING BTREE 21 | ); 22 | insert into test_e2e_mysql_db2.tbl1 values (2,'db2_tb1',20); 23 | 24 | -- tbl2_1 tbl2_2 25 | DROP TABLE IF EXISTS test_e2e_mysql_db1.tbl2_1; 26 | CREATE TABLE test_e2e_mysql_db1.tbl2_1 ( 27 | `id` int NOT NULL, 28 | `name` varchar(255) DEFAULT NULL, 29 | `age` bigint DEFAULT NULL, 30 | PRIMARY KEY (`id`) USING BTREE 31 | ); 32 | insert into test_e2e_mysql_db1.tbl2_1 values (1,'db1_tb2_1',19); 33 | 34 | DROP TABLE IF EXISTS test_e2e_mysql_db1.tbl2_2; 35 | CREATE TABLE test_e2e_mysql_db1.tbl2_2 ( 36 | `id` int NOT NULL, 37 | `name` varchar(255) DEFAULT NULL, 38 | `age` bigint DEFAULT NULL, 39 | PRIMARY KEY (`id`) USING BTREE 40 | ); 41 | insert into test_e2e_mysql_db1.tbl2_2 values (2,'db1_tb2_2',191); 42 | 43 | -- db2 44 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl2_1; 45 | CREATE TABLE test_e2e_mysql_db2.tbl2_1 ( 46 | `id` int NOT NULL, 47 | `name` varchar(255) DEFAULT NULL, 48 | `age` bigint DEFAULT NULL, 49 | PRIMARY KEY (`id`) USING BTREE 50 | ); 51 | insert into test_e2e_mysql_db2.tbl2_1 values (3,'db2_tb2_2',21); 52 | 53 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl2_2; 54 | CREATE TABLE test_e2e_mysql_db2.tbl2_2 ( 55 | `id` int NOT NULL, 56 | `name` varchar(255) DEFAULT NULL, 57 | `age` bigint DEFAULT NULL, 58 | PRIMARY KEY (`id`) USING BTREE 59 | ); 60 | insert into test_e2e_mysql_db2.tbl2_2 values (4,'db2_tbl2_2',211); 61 | 62 | 63 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl3; 64 | CREATE TABLE test_e2e_mysql_db2.tbl3 ( 65 | `id` int NOT NULL, 66 | `name` varchar(255) DEFAULT NULL, 67 | `age` bigint DEFAULT NULL, 68 | PRIMARY KEY (`id`) USING BTREE 69 | ); 70 | insert into test_e2e_mysql_db2.tbl3 values (1,'db2_tb3',22); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisMultiDbSync.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --mysql-conf database-name=test_e2e_mysql_db.* 3 | --including-tables ".*" 4 | --table-conf replication_num=1 -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisMultiDbSync_init.sql: -------------------------------------------------------------------------------- 1 | -- db1 2 | DROP DATABASE if EXISTS test_e2e_mysql_db1; 3 | CREATE DATABASE if NOT EXISTS test_e2e_mysql_db1; 4 | DROP TABLE IF EXISTS test_e2e_mysql_db1.tbl1; 5 | CREATE TABLE test_e2e_mysql_db1.tbl1 ( 6 | `id` int NOT NULL, 7 | `name` varchar(255) DEFAULT NULL, 8 | `age` bigint DEFAULT NULL, 9 | PRIMARY KEY (`id`) USING BTREE 10 | ); 11 | insert into test_e2e_mysql_db1.tbl1 values (1,'db1_tb1',18); 12 | 13 | 14 | DROP TABLE IF EXISTS test_e2e_mysql_db1.tbl2; 15 | CREATE TABLE test_e2e_mysql_db1.tbl2 ( 16 | `id` int NOT NULL, 17 | `name` varchar(255) DEFAULT NULL, 18 | `age` bigint DEFAULT NULL, 19 | PRIMARY KEY (`id`) USING BTREE 20 | ); 21 | insert into test_e2e_mysql_db1.tbl2 values (1,'db1_tb2',19); 22 | 23 | -- db2 24 | DROP DATABASE if EXISTS test_e2e_mysql_db2; 25 | CREATE DATABASE if NOT EXISTS test_e2e_mysql_db2; 26 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl1; 27 | CREATE TABLE test_e2e_mysql_db2.tbl1 ( 28 | `id` int NOT NULL, 29 | `name` varchar(255) DEFAULT NULL, 30 | `age` bigint DEFAULT NULL, 31 | PRIMARY KEY (`id`) USING BTREE 32 | ); 33 | insert into test_e2e_mysql_db2.tbl1 values (1,'db2_tb1',20); 34 | 35 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl2; 36 | CREATE TABLE test_e2e_mysql_db2.tbl2 ( 37 | `id` int NOT NULL, 38 | `name` varchar(255) DEFAULT NULL, 39 | `age` bigint DEFAULT NULL, 40 | PRIMARY KEY (`id`) USING BTREE 41 | ); 42 | insert into test_e2e_mysql_db2.tbl2 values (1,'db2_tb2',21); 43 | 44 | DROP TABLE IF EXISTS test_e2e_mysql_db2.tbl3; 45 | CREATE TABLE test_e2e_mysql_db2.tbl3 ( 46 | `id` int NOT NULL, 47 | `name` varchar(255) DEFAULT NULL, 48 | `age` bigint DEFAULT NULL, 49 | PRIMARY KEY (`id`) USING BTREE 50 | ); 51 | insert into test_e2e_mysql_db2.tbl3 values (1,'db2_tb3',22); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisSQLParse.txt: -------------------------------------------------------------------------------- 1 | mysql-sync-database 2 | --database test_e2e_mysql 3 | --mysql-conf database-name=test_e2e_mysql 4 | --including-tables "tbl.*|add_tbl" 5 | --table-conf replication_num=1 6 | --schema-change-mode sql_parser 7 | --single-sink true -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2DorisSQLParse_init.sql: -------------------------------------------------------------------------------- 1 | DROP DATABASE if EXISTS test_e2e_mysql; 2 | CREATE DATABASE if NOT EXISTS test_e2e_mysql; 3 | DROP TABLE IF EXISTS test_e2e_mysql.tbl1; 4 | CREATE TABLE test_e2e_mysql.tbl1 ( 5 | `name` varchar(256) primary key, 6 | `age` int 7 | ); 8 | insert into test_e2e_mysql.tbl1 values ('doris_1',1); 9 | 10 | 11 | DROP TABLE IF EXISTS test_e2e_mysql.tbl2; 12 | CREATE TABLE test_e2e_mysql.tbl2 ( 13 | `name` varchar(256) primary key, 14 | `age` int 15 | ); 16 | insert into test_e2e_mysql.tbl2 values ('doris_2',2); 17 | 18 | 19 | DROP TABLE IF EXISTS test_e2e_mysql.tbl3; 20 | CREATE TABLE test_e2e_mysql.tbl3 ( 21 | `name` varchar(256) primary key, 22 | `age` int 23 | ); 24 | insert into test_e2e_mysql.tbl3 values ('doris_3',3); 25 | 26 | 27 | DROP TABLE IF EXISTS test_e2e_mysql.tbl4; 28 | CREATE TABLE test_e2e_mysql.tbl4 ( 29 | `name` varchar(256) primary key, 30 | `age` int 31 | ); 32 | 33 | 34 | DROP TABLE IF EXISTS test_e2e_mysql.tbl5; 35 | CREATE TABLE test_e2e_mysql.tbl5 ( 36 | `name` varchar(256) primary key, 37 | `age` int 38 | ); 39 | insert into test_e2e_mysql.tbl5 values ('doris_5',5); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/container/e2e/mysql2doris/testMySQL2Doris_init.sql: -------------------------------------------------------------------------------- 1 | DROP DATABASE if EXISTS test_e2e_mysql; 2 | CREATE DATABASE if NOT EXISTS test_e2e_mysql; 3 | DROP TABLE IF EXISTS test_e2e_mysql.tbl1; 4 | CREATE TABLE test_e2e_mysql.tbl1 ( 5 | `name` varchar(256) primary key, 6 | `age` int 7 | ); 8 | insert into test_e2e_mysql.tbl1 values ('doris_1',1); 9 | 10 | 11 | DROP TABLE IF EXISTS test_e2e_mysql.tbl2; 12 | CREATE TABLE test_e2e_mysql.tbl2 ( 13 | `name` varchar(256) primary key, 14 | `age` int 15 | ); 16 | insert into test_e2e_mysql.tbl2 values ('doris_2',2); 17 | 18 | 19 | DROP TABLE IF EXISTS test_e2e_mysql.tbl3; 20 | CREATE TABLE test_e2e_mysql.tbl3 ( 21 | `name` varchar(256) primary key, 22 | `age` int 23 | ); 24 | insert into test_e2e_mysql.tbl3 values ('doris_3',3); 25 | 26 | 27 | DROP TABLE IF EXISTS test_e2e_mysql.tbl4; 28 | CREATE TABLE test_e2e_mysql.tbl4 ( 29 | `name` varchar(256) primary key, 30 | `age` int 31 | ); 32 | insert into test_e2e_mysql.tbl4 values ('doris_4',4); 33 | 34 | DROP TABLE IF EXISTS test_e2e_mysql.tbl5; 35 | CREATE TABLE test_e2e_mysql.tbl5 ( 36 | `name` varchar(256) primary key, 37 | `age` int 38 | ); 39 | insert into test_e2e_mysql.tbl5 values ('doris_5',5); -------------------------------------------------------------------------------- /flink-doris-connector/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | rootLogger.level = INFO 20 | rootLogger.appenderRef.test.ref = TestLogger 21 | 22 | appender.testlogger.name = TestLogger 23 | appender.testlogger.type = CONSOLE 24 | appender.testlogger.layout.type = PatternLayout 25 | appender.testlogger.layout.pattern = %d{HH:mm:ss,SSS} %-5p %-60c [%t] %x - %m%n 26 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | -------------------------------------------------------------------------------- /tools/releasing/create_release_branch.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, 13 | # software distributed under the License is distributed on an 14 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | # KIND, either express or implied. See the License for the 16 | # specific language governing permissions and limitations 17 | # under the License. 18 | 19 | ############################################################## 20 | # This script is create release branch 21 | ############################################################## 22 | 23 | RELEASE_VERSION=${RELEASE_VERSION} 24 | 25 | if [ -z "${RELEASE_VERSION}" ]; then 26 | echo "RELEASE_VERSION was not set" 27 | exit 1 28 | fi 29 | 30 | # fail immediately 31 | set -o errexit 32 | set -o nounset 33 | 34 | CURR_DIR=`pwd` 35 | BASE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )" 36 | PROJECT_ROOT="${BASE_DIR}/../../" 37 | 38 | # Sanity check to ensure that resolved paths are valid; a LICENSE file should aways exist in project root 39 | if [ ! -f ${PROJECT_ROOT}/LICENSE.txt ]; then 40 | echo "Project root path ${PROJECT_ROOT} is not valid; script may be in the wrong directory." 41 | exit 1 42 | fi 43 | 44 | TARGET_BRANCH=release-${RELEASE_VERSION} 45 | 46 | cd ${PROJECT_ROOT} 47 | git checkout -b ${TARGET_BRANCH} 48 | 49 | RELEASE_COMMIT_HASH=`git rev-parse HEAD` 50 | echo "Created a new release branch ${TARGET_BRANCH} with commit hash ${RELEASE_COMMIT_HASH}." 51 | 52 | cd ${CURR_DIR} -------------------------------------------------------------------------------- /tools/releasing/update_branch_version.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, 13 | # software distributed under the License is distributed on an 14 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | # KIND, either express or implied. See the License for the 16 | # specific language governing permissions and limitations 17 | # under the License. 18 | 19 | ############################################################## 20 | # This script is update branch version in pom.xml 21 | ############################################################## 22 | 23 | RELEASE_VERSION=${RELEASE_VERSION} 24 | 25 | if [ -z "${RELEASE_VERSION}" ]; then 26 | echo "RELEASE_VERSION was not set." 27 | exit 1 28 | fi 29 | 30 | # fail immediately 31 | set -o errexit 32 | set -o nounset 33 | 34 | CURR_DIR=`pwd` 35 | BASE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )" 36 | PROJECT_ROOT="${BASE_DIR}/../../" 37 | 38 | # Sanity check to ensure that resolved paths are valid; a LICENSE file should aways exist in project root 39 | if [ ! -f ${PROJECT_ROOT}/LICENSE.txt ]; then 40 | echo "Project root path ${PROJECT_ROOT} is not valid; script may be in the wrong directory." 41 | exit 1 42 | fi 43 | 44 | cd ${PROJECT_ROOT}/flink-doris-connector 45 | mvn versions:set -DgenerateBackupPoms=false -DnewVersion=${RELEASE_VERSION} 46 | mvn versions:set-property -DgenerateBackupPoms=false -Dproperty=revision -DnewVersion=${RELEASE_VERSION} 47 | 48 | git commit -am "[release] Update version to ${RELEASE_VERSION}" 49 | 50 | RELEASE_VERSION_COMMIT_HASH=`git rev-parse HEAD` 51 | 52 | echo "Done. Created a new commit for the new version ${RELEASE_VERSION}, with hash ${RELEASE_VERSION_COMMIT_HASH}" 53 | echo "If this is a new version to be released (or a candidate to be voted on), don't forget to create a signed release tag on GitHub and push the changes." 54 | 55 | cd ${CURR_DIR} --------------------------------------------------------------------------------