Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-8345

Oracle connector: archive.log.only.mode stop working after reach SYSDATE SCN

XMLWordPrintable

      In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      2.7.1.Final

      What is the connector configuration?

       

      connector.class = io.debezium.connector.oracle.OracleConnector
      transforms = unwrap
      schema.include.list = ${env}
      schema.history.internal.pubsub.project.id = ${env}
      transforms.unwrap.drop.fields = before
      record.processing.threads =
      errors.retry.delay.initial.ms = 60000
      transforms.unwrap.type = io.debezium.transforms.ExtractNewRecordState
      log.mining.archive.log.only.mode = true
      offset.storage.pubsub.ordering.enabled = false
      key.converter = org.apache.kafka.connect.json.JsonConverter
      schema.history.internal.pubsub.ordering.enabled = false
      log.mining.scn.gap.detection.gap.size.min = 1000000
      database.dbname = ${env}
      database.user = ${env}
      offset.storage = org.apache.kafka.connect.storage.FileOffsetBackingStore
      log.log.mining.archive.log.only.scn.poll.interval.ms = 10000
      internal.task.management.timeout.ms = 180000
      errors.max.retries = -1
      offset.storage.pubsub.retry.max.rpc.timeout.ms = 30000
      database.password = ${env}
      name = pubsub
      max.batch.size = 2048
      log.mining.scn.gap.detection.time.interval.max.ms = 20000
      snapshot.mode = NO_DATA
      max.queue.size = 8129
      transforms.unwrap.delete.handling.mode = rewrite
      record.processing.shutdown.timeout.ms = 1000
      log.mining.transaction.retention.ms = 0
      log.mining.strategy = online_catalog
      schema.history.internal.store.only.captured.tables.ddl = true
      schema.history.internal.store.only.captured.databases.ddl = true
      record.processing.order = ORDERED
      schema.history.internal.file.filename = /debezium/data/FileSchemaHistory.dat
      tombstones.on.delete = false
      topic.prefix = ${env}
      decimal.handling.mode = double
      offset.storage.file.filename = /debezium/data/offsets.dat
      offset.storage.pubsub.project.id = ${env}
      value.converter = org.apache.kafka.connect.json.JsonConverter
      database.event.processing.failure.handling.mode = warn
      schema.history.internal.pubsub.retry.max.rpc.timeout.ms = 30000
      database.server.id = 8
      log.mining.batch.size.max = 200000
      database.pdb.name = ${env}
      database.url = ${env}
      errors.retry.delay.max.ms = 3840000
      offset.flush.timeout.ms = 5000
      database.port = ${env}
      offset.flush.interval.ms = 60000
      schema.history.internal = io.debezium.storage.file.history.FileSchemaHistory
      record.processing.with.serial.consumer = false
      log.mining.batch.size.min = 10000
      log.mining.batch.size.default = 50000
      transforms.unwrap.add.fields = source.ts_ms:ts_ms, op:op
      table.include.list = ${env}

       

       

      What is the captured database version and mode of deployment?

      (E.g. on-premises, with a specific cloud provider, etc.)

      Oracle 19c on-premises

      What behavior do you expect?

      Debezium streaming shouldn't stopped or error when reach latest scn.

      What behavior do you see?

      Debezium streaming stopped after reach latest scn and after some retry the connector is gracefully closed.

      Community thread

      Do you see the same behaviour using the latest released Debezium version?

      (Ideally, also verify with latest Alpha/Beta/CR version)

      Haven't verified with latest version (3.0.Final) yet, but tried v2.7.3.Final and the issue still existed.

      Do you have the connector logs, ideally from start till finish?

      (You might be asked later to provide DEBUG/TRACE level log)

      you can find full log here community thread

      2024-10-17T16:41:27.576Z|2|DEBUG|cdc|,,,|1|poll|io.debezium.connector.base.ChangeEventQueue|checking for more records...2024-10-17T16:41:27.576Z|2|DEBUG|cdc|,,,|1|poll|io.debezium.connector.base.ChangeEventQueue|polling records...2024-10-17T16:41:27.576Z|2|DEBUG|cdc|,,,|1|poll|io.debezium.connector.base.ChangeEventQueue|no records available or batch size not reached yet, sleeping a bit...2024-10-17T16:41:27.805Z|2|ERROR|cdc|,,,|1|execute|io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource|Mining session stopped due to error.: io.debezium.DebeziumException: None of the log files contain offset SCN: 14767143302342, re-snapshot is required.    at io.debezium.connector.oracle.logminer.LogFileCollector.getLogs(LogFileCollector.java:95)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.isStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1076)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.waitForStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1050)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:175)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)    at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:312)    at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)    at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)    at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)    at java.base/java.lang.Thread.run(Thread.java:829)
      2024-10-17T16:41:27.805Z|2|ERROR|cdc|,,,|1|setProducerThrowable|io.debezium.pipeline.ErrorHandler|Producer failure: io.debezium.DebeziumException: None of the log files contain offset SCN: 14767143302342, re-snapshot is required.    at io.debezium.connector.oracle.logminer.LogFileCollector.getLogs(LogFileCollector.java:95)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.isStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1076)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.waitForStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1050)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:175)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)    at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:312)    at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)    at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)    at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)    at java.base/java.lang.Thread.run(Thread.java:829)
      2024-10-17T16:41:27.805Z|2|INFO|cdc|,,,|1|execute|io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource|startScn=14767143302342, endScn=null2024-10-17T16:41:27.850Z|2|INFO|cdc|,,,|1|execute|io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource|Streaming metrics dump: LogMinerStreamingChangeEventSourceMetrics{connectorConfig=io.debezium.connector.oracle.OracleConnectorConfig@3e2a026, startTime=2024-10-17T16:40:22.750766Z, clock=SystemClock[Z], currentScn=null, offsetScn=null, commitScn=null, oldestScn=null, oldestScnTime=null, currentLogFileNames=[], redoLogStatuses=[], databaseZoneOffset=Z, batchSize=50000, logSwitchCount=0, logMinerQueryCount=0, sleepTime=1000, minimumLogsMined=0, maximumLogsMined=0, maxBatchProcessingThroughput=0, timeDifference=0, processedRowsCount=0, activeTransactionCount=0, rolledBackTransactionCount=0, oversizedTransactionCount=0, changesCount=0, scnFreezeCount=0, batchProcessingDuration={min=PT0S,max=PT0S,total=PT0S}, fetchQueryDuration={min=PT0S,max=PT0S,total=PT0S}, commitDuration={min=PT0S,max=PT0S,total=PT0S}, lagFromSourceDuration={min=PT0S,max=PT0S,total=PT0S}, miningSessionStartupDuration={min=PT0S,max=PT0S,total=PT0S}, parseTimeDuration={min=PT0S,max=PT0S,total=PT0S}, resultSetNextDuration={min=PT0S,max=PT0S,total=PT0S}, userGlobalAreaMemory={value=0,max=0}, processGlobalAreaMemory={value=0,max=0}, abandonedTransactionIds=[], rolledBackTransactionIds=[]} 2024-10-17T16:41:27.850Z|2|INFO|cdc|,,,|1|execute|io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource|Offsets: OracleOffsetContext [scn=14767143302342, commit_scn=[], lcr_position=null]2024-10-17T16:41:27.850Z|2|INFO|cdc|,,,|1|streamEvents|io.debezium.pipeline.ChangeEventSourceCoordinator|Finished streaming2024-10-17T16:41:27.850Z|2|INFO|cdc|,,,|1|streamingConnected|io.debezium.pipeline.ChangeEventSourceCoordinator|Connected metrics set to 'false'2024-10-17T16:41:28.076Z|2|DEBUG|cdc|,,,|1|poll|io.debezium.connector.base.ChangeEventQueue|checking for more records...2024-10-17T16:41:28.077Z|2|ERROR|cdc|,,,|1|closeEngineWithException|io.debezium.embedded.async.AsyncEmbeddedEngine|Engine has failed with : java.util.concurrent.ExecutionException: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.    at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)    at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)    at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:476)    at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:212)    at io.debezium.server.DebeziumServer.lambda$start$1(DebeziumServer.java:178)    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)    at java.base/java.lang.Thread.run(Thread.java:829)Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.    at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:264)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)    at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:312)    at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)    at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)    at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)    ... 3 moreCaused by: io.debezium.DebeziumException: None of the log files contain offset SCN: 14767143302342, re-snapshot is required.    at io.debezium.connector.oracle.logminer.LogFileCollector.getLogs(LogFileCollector.java:95)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.isStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1076)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.waitForStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1050)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:175)    ... 9 more
      2024-10-17T16:41:28.078Z|2|DEBUG|cdc|,,,|1|closeEngineWithException|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping io.debezium.embedded.async.AsyncEmbeddedEngine2024-10-17T16:41:28.078Z|2|INFO|cdc|,,,|1|setEngineState|io.debezium.embedded.async.AsyncEmbeddedEngine|Engine state has changed from 'POLLING_TASKS' to 'STOPPING'2024-10-17T16:41:28.078Z|2|DEBUG|cdc|,,,|1|stopConnector|io.debezium.embedded.async.AsyncEmbeddedEngine|Tasks were already started, stopping record service and tasks.2024-10-17T16:41:28.078Z|2|DEBUG|cdc|,,,|1|stopRecordService|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping records service.2024-10-17T16:41:28.078Z|2|DEBUG|cdc|,,,|1|stopSourceTasks|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping source connector tasks.2024-10-17T16:41:28.078Z|2|DEBUG|cdc|,,,|1|stopSourceTasks|io.debezium.embedded.async.AsyncEmbeddedEngine|Waiting max. for 180000 ms for individual source tasks to stop.2024-10-17T16:41:28.078Z|2|DEBUG|cdc|,,,|1|lambda$stopSourceTasks$3|io.debezium.embedded.async.AsyncEmbeddedEngine|Committing task's offset.2024-10-17T16:41:28.079Z|2|DEBUG|cdc|,,,|1|lambda$stopSourceTasks$3|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping Connect task.2024-10-17T16:41:28.079Z|2|INFO|cdc|,,,|1|stop|io.debezium.connector.common.BaseSourceTask|Stopping down connector2024-10-17T16:41:28.080Z|2|INFO|cdc|,,,|1|stop|io.debezium.pipeline.signal.SignalProcessor|SignalProcessor stopped2024-10-17T16:41:28.080Z|2|INFO|cdc|,,,|1|close|io.debezium.service.DefaultServiceRegistry|Debezium ServiceRegistry stopped.2024-10-17T16:41:28.091Z|2|INFO|cdc|,,,|1|lambda$doClose$4|io.debezium.jdbc.JdbcConnection|Connection gracefully closed2024-10-17T16:41:28.095Z|2|INFO|cdc|,,,|1|lambda$doClose$4|io.debezium.jdbc.JdbcConnection|Connection gracefully closed2024-10-17T16:41:28.095Z|2|DEBUG|cdc|,,,|1|setTaskState|io.debezium.connector.common.BaseSourceTask|Setting task state to 'STOPPED', previous state was 'RUNNING'2024-10-17T16:41:28.095Z|2|INFO|cdc|,,,|1|stopSourceTasks|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopped task #1 out of 1 tasks (it took 16 ms to stop the task).2024-10-17T16:41:28.095Z|2|DEBUG|cdc|,,,|1|stopSourceTasks|io.debezium.embedded.async.AsyncEmbeddedEngine|Calling connector callback after task is stopped.2024-10-17T16:41:28.096Z|2|DEBUG|cdc|,,,|1|taskStopped|io.debezium.server.ConnectorLifecycle|Task stopped2024-10-17T16:41:28.096Z|2|DEBUG|cdc|,,,|1|stopSourceTasks|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping all remaining tasks if there are any.2024-10-17T16:41:28.096Z|2|DEBUG|cdc|,,,|1|stopOffsetStore|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping offset backing store.2024-10-17T16:41:28.096Z|2|INFO|cdc|,,,|1|stop|org.apache.kafka.connect.storage.FileOffsetBackingStore|Stopped FileOffsetBackingStore2024-10-17T16:41:28.096Z|2|DEBUG|cdc|,,,|1|stopConnector|io.debezium.embedded.async.AsyncEmbeddedEngine|Stopping the connector.2024-10-17T16:41:28.096Z|2|DEBUG|cdc|,,,|1|stopConnector|io.debezium.embedded.async.AsyncEmbeddedEngine|Calling connector callback after connector stop2024-10-17T16:41:28.096Z|2|DEBUG|cdc|,,,|1|connectorStopped|io.debezium.server.ConnectorLifecycle|Connector stopped2024-10-17T16:41:28.097Z|2|INFO|cdc|,,,|1|finishShutDown|io.debezium.embedded.async.AsyncEmbeddedEngine|Engine is stopped.2024-10-17T16:41:28.097Z|2|INFO|cdc|,,,|1|setEngineState|io.debezium.embedded.async.AsyncEmbeddedEngine|Engine state has changed from 'STOPPING' to 'STOPPED'2024-10-17T16:41:28.097Z|2|DEBUG|cdc|,,,|1|finishShutDown|io.debezium.embedded.async.AsyncEmbeddedEngine|Calling completion handler.2024-10-17T16:41:28.097Z|2|ERROR|cdc|,,,|1|handle|io.debezium.server.ConnectorLifecycle|Connector completed: success = 'false', message = 'org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.', error = 'org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.': org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.    at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:264)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:62)    at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:312)    at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)    at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)    at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)    at java.base/java.lang.Thread.run(Thread.java:829)Caused by: io.debezium.DebeziumException: None of the log files contain offset SCN: 14767143302342, re-snapshot is required.    at io.debezium.connector.oracle.logminer.LogFileCollector.getLogs(LogFileCollector.java:95)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.isStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1076)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.waitForStartScnInArchiveLogs(LogMinerStreamingChangeEventSource.java:1050)    at io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource.execute(LogMinerStreamingChangeEventSource.java:175)    ... 9 more
      2024-10-17T16:41:28.110Z|2|INFO|cdc|,,,|1|stop|io.debezium.server.DebeziumServer|Received request to stop the engine2024-10-17T16:41:28.110Z|2|DEBUG|cdc|,,,|1|close|io.debezium.embedded.async.AsyncEmbeddedEngine|Engine shutdown called.2024-10-17T16:41:28.111Z|2|ERROR|cdc|,,,|1|stop|io.debezium.server.DebeziumServer|Exception while shuttting down Debezium [Error Occurred After Shutdown]: java.lang.IllegalStateException: Engine has been already shut down.    at io.debezium.embedded.async.AsyncEmbeddedEngine.close(AsyncEmbeddedEngine.java:249)    at io.debezium.server.DebeziumServer.stop(DebeziumServer.java:249)    at io.debezium.server.DebeziumServer_Observer_stop_d470e05516c8d86f955c7275c00cebdc903562ab.notify(Unknown Source)    at io.quarkus.arc.impl.EventImpl$Notifier.notifyObservers(EventImpl.java:346)    at io.quarkus.arc.impl.EventImpl$Notifier.notify(EventImpl.java:328)    at io.quarkus.arc.impl.EventImpl.fire(EventImpl.java:82)    at io.quarkus.arc.runtime.ArcRecorder.fireLifecycleEvent(ArcRecorder.java:155)    at io.quarkus.arc.runtime.ArcRecorder$2.run(ArcRecorder.java:111)    at io.quarkus.runtime.StartupContext.runAllInReverseOrder(StartupContext.java:84)    at io.quarkus.runtime.StartupContext.close(StartupContext.java:73)    at io.quarkus.runner.ApplicationImpl.doStop(Unknown Source)    at io.quarkus.runtime.Application.stop(Application.java:208)    at io.quarkus.runtime.Application.stop(Application.java:155)    at io.quarkus.runtime.ApplicationLifecycleManager.run(ApplicationLifecycleManager.java:227)    at io.quarkus.runtime.Quarkus.run(Quarkus.java:71)    at io.quarkus.runtime.Quarkus.run(Quarkus.java:44)    at io.quarkus.runtime.Quarkus.run(Quarkus.java:124)    at io.debezium.server.Main.main(Main.java:15)
      2024-10-17T16:41:28.115Z|2|DEBUG|cdc|,,,|1|log|io.grpc.netty.shaded.io.netty.util.internal.logging.AbstractInternalLogger|[id: 0x82ab05b5, L:/10.133.10.249:55666 - R:pubsub.googleapis.com/142.250.199.10:443] OUTBOUND GO_AWAY: lastStreamId=0 errorCode=0 length=0 bytes=2024-10-17T16:41:28.130Z|2|DEBUG|cdc|,,,|1|free|io.netty.buffer.PoolThreadCache|Freed 3 thread-local buffer(s) from thread: vert.x-eventloop-thread-12024-10-17T16:41:28.130Z|2|DEBUG|cdc|,,,|1|free|io.netty.buffer.PoolThreadCache|Freed 3 thread-local buffer(s) from thread: vert.x-eventloop-thread-0Oct 17, 2024 4:41:28 PM io.quarkus.bootstrap.runner.Timing printStopTimeINFO: debezium-server-dist stopped in 0.034s

       

      How to reproduce the issue using our tutorial deployment?

      By using  this config on oracle connector

      log.mining.archive.log.only.mode = true 

      Feature request or enhancement

      For feature requests or enhancements, provide this information, please:

      Which use case/requirement will be addressed by the proposed feature?

      <Your answer>

      Implementation ideas (optional)

      <Your answer>

              ccranfor@redhat.com Chris Cranford
              tanawat.j tanawat jirawattanakul
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: