Skip to content
Projects
Groups
Snippets
Help
This project
Loading...
Sign in / Register
Toggle navigation
C
cdc-engine2
Overview
Overview
Details
Activity
Cycle Analytics
Repository
Repository
Files
Commits
Branches
Tags
Contributors
Graph
Compare
Charts
Issues
0
Issues
0
List
Board
Labels
Milestones
Merge Requests
0
Merge Requests
0
CI / CD
CI / CD
Pipelines
Jobs
Schedules
Charts
Wiki
Wiki
Snippets
Snippets
Members
Members
Collapse sidebar
Close sidebar
Activity
Graph
Charts
Create a new issue
Jobs
Commits
Issue Boards
Open sidebar
黄营
cdc-engine2
Commits
5fe6d122
Commit
5fe6d122
authored
Feb 21, 2025
by
成浩
Browse files
Options
Browse Files
Download
Email Patches
Plain Diff
增加针对oracle 西语环境捕获中文乱码的处理方法
parent
2f93df9e
Expand all
Hide whitespace changes
Inline
Side-by-side
Showing
6 changed files
with
347 additions
and
1 deletions
+347
-1
BaseSourceTask.java
.../main/java/com/tbyf/cdcengine2/oracle/BaseSourceTask.java
+0
-0
CustomOracleConnector.java
...ava/com/tbyf/cdcengine2/oracle/CustomOracleConnector.java
+86
-0
CustomOracleConnectorTask.java
...com/tbyf/cdcengine2/oracle/CustomOracleConnectorTask.java
+185
-0
OracleCdcEngine.java
...main/java/com/tbyf/cdcengine2/oracle/OracleCdcEngine.java
+3
-1
OracleEventMetadataProvider.java
...m/tbyf/cdcengine2/oracle/OracleEventMetadataProvider.java
+53
-0
Provider.java
...dc/src/main/java/com/tbyf/cdcengine2/oracle/Provider.java
+20
-0
No files found.
oracle-cdc/src/main/java/com/tbyf/cdcengine2/oracle/BaseSourceTask.java
0 → 100644
View file @
5fe6d122
This diff is collapsed.
Click to expand it.
oracle-cdc/src/main/java/com/tbyf/cdcengine2/oracle/CustomOracleConnector.java
0 → 100644
View file @
5fe6d122
package
com
.
tbyf
.
cdcengine2
.
oracle
;
import
io.debezium.config.Configuration
;
import
io.debezium.connector.common.RelationalBaseSourceConnector
;
import
io.debezium.connector.oracle.Module
;
import
io.debezium.connector.oracle.OracleConnection
;
import
io.debezium.connector.oracle.OracleConnectorConfig
;
import
io.debezium.connector.oracle.OracleConnectorTask
;
import
io.debezium.relational.RelationalDatabaseConnectorConfig
;
import
org.apache.kafka.connect.connector.Task
;
import
org.apache.kafka.common.config.ConfigDef
;
import
org.apache.kafka.common.config.ConfigValue
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
import
java.sql.SQLException
;
import
java.util.Collections
;
import
java.util.HashMap
;
import
java.util.List
;
import
java.util.Map
;
public
class
CustomOracleConnector
extends
RelationalBaseSourceConnector
{
private
static
final
Logger
LOGGER
=
LoggerFactory
.
getLogger
(
CustomOracleConnector
.
class
);
private
Map
<
String
,
String
>
properties
;
@Override
public
String
version
()
{
return
Module
.
version
();
}
@Override
public
void
start
(
Map
<
String
,
String
>
props
)
{
this
.
properties
=
Collections
.
unmodifiableMap
(
new
HashMap
<>(
props
));
}
@Override
public
Class
<?
extends
Task
>
taskClass
()
{
return
CustomOracleConnectorTask
.
class
;
}
@Override
public
List
<
Map
<
String
,
String
>>
taskConfigs
(
int
maxTasks
)
{
if
(
maxTasks
>
1
)
{
throw
new
IllegalArgumentException
(
"Only a single connector task may be started"
);
}
return
Collections
.
singletonList
(
properties
);
}
@Override
public
void
stop
()
{
}
@Override
public
ConfigDef
config
()
{
return
OracleConnectorConfig
.
configDef
();
}
@Override
protected
void
validateConnection
(
Map
<
String
,
ConfigValue
>
configValues
,
Configuration
config
)
{
final
ConfigValue
databaseValue
=
configValues
.
get
(
RelationalDatabaseConnectorConfig
.
DATABASE_NAME
.
name
());
if
(!
databaseValue
.
errorMessages
().
isEmpty
())
{
return
;
}
final
ConfigValue
hostnameValue
=
configValues
.
get
(
RelationalDatabaseConnectorConfig
.
HOSTNAME
.
name
());
final
ConfigValue
userValue
=
configValues
.
get
(
RelationalDatabaseConnectorConfig
.
USER
.
name
());
OracleConnectorConfig
connectorConfig
=
new
OracleConnectorConfig
(
config
);
try
(
OracleConnection
connection
=
new
OracleConnection
(
connectorConfig
.
getJdbcConfig
(),
()
->
getClass
().
getClassLoader
()))
{
LOGGER
.
debug
(
"Successfully tested connection for {} with user '{}'"
,
OracleConnection
.
connectionString
(
connectorConfig
.
getJdbcConfig
()),
connection
.
username
());
}
catch
(
SQLException
|
RuntimeException
e
)
{
LOGGER
.
error
(
"Failed testing connection for {} with user '{}'"
,
config
.
withMaskedPasswords
(),
userValue
,
e
);
hostnameValue
.
addErrorMessage
(
"Unable to connect: "
+
e
.
getMessage
());
}
}
@Override
protected
Map
<
String
,
ConfigValue
>
validateAllFields
(
Configuration
config
)
{
return
config
.
validate
(
OracleConnectorConfig
.
ALL_FIELDS
);
}
}
oracle-cdc/src/main/java/com/tbyf/cdcengine2/oracle/CustomOracleConnectorTask.java
0 → 100644
View file @
5fe6d122
package
com
.
tbyf
.
cdcengine2
.
oracle
;
import
io.debezium.DebeziumException
;
import
io.debezium.config.Configuration
;
import
io.debezium.config.Field
;
import
io.debezium.connector.base.ChangeEventQueue
;
import
io.debezium.connector.oracle.*
;
import
io.debezium.jdbc.JdbcConfiguration
;
import
io.debezium.pipeline.ChangeEventSourceCoordinator
;
import
io.debezium.pipeline.DataChangeEvent
;
import
io.debezium.pipeline.ErrorHandler
;
import
io.debezium.pipeline.EventDispatcher
;
import
io.debezium.pipeline.spi.Offsets
;
import
io.debezium.relational.TableId
;
import
io.debezium.schema.TopicSelector
;
import
io.debezium.util.Clock
;
import
io.debezium.util.SchemaNameAdjuster
;
import
org.apache.kafka.connect.source.SourceRecord
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
import
java.sql.SQLException
;
import
java.util.Collections
;
import
java.util.List
;
import
java.util.stream.Collectors
;
public
class
CustomOracleConnectorTask
extends
BaseSourceTask
<
OraclePartition
,
OracleOffsetContext
>
{
private
static
final
Logger
LOGGER
=
LoggerFactory
.
getLogger
(
CustomOracleConnectorTask
.
class
);
private
static
final
String
CONTEXT_NAME
=
"custome-oracle-connector-task"
;
private
volatile
OracleTaskContext
taskContext
;
private
volatile
ChangeEventQueue
<
DataChangeEvent
>
queue
;
private
volatile
OracleConnection
jdbcConnection
;
private
volatile
ErrorHandler
errorHandler
;
private
volatile
OracleDatabaseSchema
schema
;
@Override
public
String
version
()
{
return
Module
.
version
();
}
@Override
public
ChangeEventSourceCoordinator
<
OraclePartition
,
OracleOffsetContext
>
start
(
Configuration
config
)
{
OracleConnectorConfig
connectorConfig
=
new
OracleConnectorConfig
(
config
);
TopicSelector
<
TableId
>
topicSelector
=
OracleTopicSelector
.
defaultSelector
(
connectorConfig
);
SchemaNameAdjuster
schemaNameAdjuster
=
connectorConfig
.
schemaNameAdjustmentMode
().
createAdjuster
();
JdbcConfiguration
jdbcConfig
=
connectorConfig
.
getJdbcConfig
();
jdbcConfig
=
JdbcConfiguration
.
adapt
(
jdbcConfig
.
edit
()
.
with
(
JdbcConfiguration
.
CONNECTION_FACTORY_CLASS
,
config
.
getString
(
JdbcConfiguration
.
CONNECTION_FACTORY_CLASS
))
.
build
());
//将上面代码重新赋值给jdbcConfig变量
jdbcConnection
=
new
OracleConnection
(
jdbcConfig
,
()
->
getClass
().
getClassLoader
());
validateRedoLogConfiguration
();
OracleValueConverters
valueConverters
=
new
OracleValueConverters
(
connectorConfig
,
jdbcConnection
);
OracleDefaultValueConverter
defaultValueConverter
=
new
OracleDefaultValueConverter
(
valueConverters
,
jdbcConnection
);
StreamingAdapter
.
TableNameCaseSensitivity
tableNameCaseSensitivity
=
connectorConfig
.
getAdapter
().
getTableNameCaseSensitivity
(
jdbcConnection
);
this
.
schema
=
new
OracleDatabaseSchema
(
connectorConfig
,
valueConverters
,
defaultValueConverter
,
schemaNameAdjuster
,
topicSelector
,
tableNameCaseSensitivity
);
Offsets
<
OraclePartition
,
OracleOffsetContext
>
previousOffsets
=
getPreviousOffsets
(
new
Provider
(
connectorConfig
),
connectorConfig
.
getAdapter
().
getOffsetContextLoader
());
OraclePartition
partition
=
previousOffsets
.
getTheOnlyPartition
();
OracleOffsetContext
previousOffset
=
previousOffsets
.
getTheOnlyOffset
();
validateAndLoadDatabaseHistory
(
connectorConfig
,
partition
,
previousOffset
,
schema
);
taskContext
=
new
OracleTaskContext
(
connectorConfig
,
schema
);
Clock
clock
=
Clock
.
system
();
// Set up the task record queue ...
this
.
queue
=
new
ChangeEventQueue
.
Builder
<
DataChangeEvent
>()
.
pollInterval
(
connectorConfig
.
getPollInterval
())
.
maxBatchSize
(
connectorConfig
.
getMaxBatchSize
())
.
maxQueueSize
(
connectorConfig
.
getMaxQueueSize
())
.
loggingContextSupplier
(()
->
taskContext
.
configureLoggingContext
(
CONTEXT_NAME
))
.
build
();
errorHandler
=
new
OracleErrorHandler
(
connectorConfig
,
queue
);
final
OracleEventMetadataProvider
metadataProvider
=
new
OracleEventMetadataProvider
();
EventDispatcher
<
OraclePartition
,
TableId
>
dispatcher
=
new
EventDispatcher
<>(
connectorConfig
,
topicSelector
,
schema
,
queue
,
connectorConfig
.
getTableFilters
().
dataCollectionFilter
(),
DataChangeEvent:
:
new
,
metadataProvider
,
schemaNameAdjuster
);
final
OracleStreamingChangeEventSourceMetrics
streamingMetrics
=
new
OracleStreamingChangeEventSourceMetrics
(
taskContext
,
queue
,
metadataProvider
,
connectorConfig
);
ChangeEventSourceCoordinator
<
OraclePartition
,
OracleOffsetContext
>
coordinator
=
new
ChangeEventSourceCoordinator
<>(
previousOffsets
,
errorHandler
,
OracleConnector
.
class
,
connectorConfig
,
new
OracleChangeEventSourceFactory
(
connectorConfig
,
jdbcConnection
,
errorHandler
,
dispatcher
,
clock
,
schema
,
jdbcConfig
,
taskContext
,
streamingMetrics
),
new
OracleChangeEventSourceMetricsFactory
(
streamingMetrics
),
dispatcher
,
schema
);
coordinator
.
start
(
taskContext
,
this
.
queue
,
metadataProvider
);
return
coordinator
;
}
@Override
public
List
<
SourceRecord
>
doPoll
()
throws
InterruptedException
{
List
<
DataChangeEvent
>
records
=
queue
.
poll
();
List
<
SourceRecord
>
sourceRecords
=
records
.
stream
()
.
map
(
DataChangeEvent:
:
getRecord
)
.
collect
(
Collectors
.
toList
());
return
sourceRecords
;
}
@Override
public
void
doStop
()
{
try
{
if
(
jdbcConnection
!=
null
)
{
jdbcConnection
.
close
();
}
}
catch
(
SQLException
e
)
{
LOGGER
.
error
(
"Exception while closing JDBC connection"
,
e
);
}
schema
.
close
();
}
@Override
protected
Iterable
<
Field
>
getAllConfigurationFields
()
{
return
OracleConnectorConfig
.
ALL_FIELDS
;
}
private
void
validateRedoLogConfiguration
()
{
// Check whether the archive log is enabled.
// final boolean archivelogMode = jdbcConnection.isArchiveLogMode();
// if (!archivelogMode) {
// throw new DebeziumException("The Oracle server is not configured to use a archive log LOG_MODE, which is "
// + "required for this connector to work properly. Change the Oracle configuration to use a "
// + "LOG_MODE=ARCHIVELOG and restart the connector.");
// }
}
private
void
validateAndLoadDatabaseHistory
(
OracleConnectorConfig
config
,
OraclePartition
partition
,
OracleOffsetContext
offset
,
OracleDatabaseSchema
schema
)
{
if
(
offset
==
null
)
{
if
(
config
.
getSnapshotMode
().
shouldSnapshotOnSchemaError
())
{
// We are in schema only recovery mode, use the existing redo log position
// would like to also verify redo log position exists, but it defaults to 0 which is technically valid
throw
new
DebeziumException
(
"Could not find existing redo log information while attempting schema only recovery snapshot"
);
}
LOGGER
.
info
(
"Connector started for the first time, database history recovery will not be executed"
);
schema
.
initializeStorage
();
return
;
}
if
(!
schema
.
historyExists
())
{
LOGGER
.
warn
(
"Database history was not found but was expected"
);
if
(
config
.
getSnapshotMode
().
shouldSnapshotOnSchemaError
())
{
LOGGER
.
info
(
"The db-history topic is missing but we are in {} snapshot mode. "
+
"Attempting to snapshot the current schema and then begin reading the redo log from the last recorded offset."
,
OracleConnectorConfig
.
SnapshotMode
.
SCHEMA_ONLY_RECOVERY
);
}
else
{
throw
new
DebeziumException
(
"The db history topic is missing. You may attempt to recover it by reconfiguring the connector to "
+
OracleConnectorConfig
.
SnapshotMode
.
SCHEMA_ONLY_RECOVERY
);
}
schema
.
initializeStorage
();
return
;
}
schema
.
recover
(
Offsets
.
of
(
partition
,
offset
));
}
}
oracle-cdc/src/main/java/com/tbyf/cdcengine2/oracle/OracleCdcEngine.java
View file @
5fe6d122
...
...
@@ -18,9 +18,11 @@ public class OracleCdcEngine extends AbstractCdcEngine<OracleCdcEngine> {
public
OracleCdcEngine
(
OracleCdcEngineProperties
props
)
{
super
(
props
);
debeziumProps
.
setProperty
(
Constants
.
CONNECTOR_CLASS_PROP
,
OracleConnector
.
class
.
getName
());
debeziumProps
.
setProperty
(
Constants
.
CONNECTOR_CLASS_PROP
,
Custom
OracleConnector
.
class
.
getName
());
debeziumProps
.
setProperty
(
Constants
.
DATABASE_DBNAME_PROP
,
props
.
getDbname
());
debeziumProps
.
setProperty
(
"log.mining.strategy"
,
"online_catalog"
);
//要实现针对西语环境的oracle日志乱码处理
// debeziumProps.setProperty("connection.factory.class", "com.tbyf.hip.cloud.service.config.CustomConnectionFactory");
String
schema
=
props
.
getUsername
().
toUpperCase
();
if
(
props
.
getSchemaList
()!=
null
){
...
...
oracle-cdc/src/main/java/com/tbyf/cdcengine2/oracle/OracleEventMetadataProvider.java
0 → 100644
View file @
5fe6d122
package
com
.
tbyf
.
cdcengine2
.
oracle
;
import
io.debezium.pipeline.source.spi.EventMetadataProvider
;
import
io.debezium.pipeline.spi.OffsetContext
;
import
io.debezium.schema.DataCollectionId
;
import
io.debezium.util.Collect
;
import
org.apache.kafka.connect.data.Struct
;
import
java.time.Instant
;
import
java.util.Map
;
class
OracleEventMetadataProvider
implements
EventMetadataProvider
{
OracleEventMetadataProvider
()
{
}
public
Instant
getEventTimestamp
(
DataCollectionId
source
,
OffsetContext
offset
,
Object
key
,
Struct
value
)
{
if
(
value
==
null
)
{
return
null
;
}
else
{
Struct
sourceInfo
=
value
.
getStruct
(
"source"
);
if
(
source
==
null
)
{
return
null
;
}
else
{
Long
timestamp
=
sourceInfo
.
getInt64
(
"ts_ms"
);
return
timestamp
==
null
?
null
:
Instant
.
ofEpochMilli
(
timestamp
);
}
}
}
public
Map
<
String
,
String
>
getEventSourcePosition
(
DataCollectionId
source
,
OffsetContext
offset
,
Object
key
,
Struct
value
)
{
if
(
value
==
null
)
{
return
null
;
}
else
{
Struct
sourceInfo
=
value
.
getStruct
(
"source"
);
if
(
source
==
null
)
{
return
null
;
}
else
{
String
scn
=
sourceInfo
.
getString
(
"scn"
);
return
Collect
.
hashMapOf
(
"scn"
,
scn
==
null
?
"null"
:
scn
);
}
}
}
public
String
getTransactionId
(
DataCollectionId
source
,
OffsetContext
offset
,
Object
key
,
Struct
value
)
{
if
(
value
==
null
)
{
return
null
;
}
else
{
Struct
sourceInfo
=
value
.
getStruct
(
"source"
);
return
source
==
null
?
null
:
sourceInfo
.
getString
(
"txId"
);
}
}
}
oracle-cdc/src/main/java/com/tbyf/cdcengine2/oracle/Provider.java
0 → 100644
View file @
5fe6d122
package
com
.
tbyf
.
cdcengine2
.
oracle
;
import
io.debezium.connector.oracle.OracleConnectorConfig
;
import
io.debezium.connector.oracle.OraclePartition
;
import
io.debezium.pipeline.spi.Partition
;
import
java.util.Collections
;
import
java.util.Set
;
public
class
Provider
implements
Partition
.
Provider
<
OraclePartition
>
{
private
final
OracleConnectorConfig
connectorConfig
;
Provider
(
OracleConnectorConfig
connectorConfig
)
{
this
.
connectorConfig
=
connectorConfig
;
}
public
Set
<
OraclePartition
>
getPartitions
()
{
return
Collections
.
singleton
(
new
OraclePartition
(
this
.
connectorConfig
.
getLogicalName
()));
}
}
Write
Preview
Markdown
is supported
0%
Try again
or
attach a new file
Attach a file
Cancel
You are about to add
0
people
to the discussion. Proceed with caution.
Finish editing this message first!
Cancel
Please
register
or
sign in
to comment