From 34c9b2b7862ced681ab8b56868523b971a17361d Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sun, 5 Oct 2025 06:08:02 +0000 Subject: [PATCH 01/53] (#11) Debezium Engine --- conf/jvm.conf | 24 + .../config/register-postgres.json.template | 18 +- develop/docker-compose.yml | 7 +- develop/docker-monitor-compose.yml | 40 ++ develop/example/sql/dss.ri | 95 ++- develop/example/sql/pg_replica.sql | 15 + .../dashboards/sink-server.json | 355 ++++++++++- .../images/prometheus/prometheus_local.yml | 9 + develop/scripts/install.sh | 1 + pixels-sink | 45 ++ pom.xml | 81 +-- .../pixels/sink/PixelsDebeziumConsumer.java | 142 +++++ .../pixelsdb/pixels/sink/PixelsSinkApp.java | 57 +- .../pixels/sink/PixelsSinkProvider.java | 42 +- .../pixels/sink/concurrent/SinkContext.java | 154 ++++- .../concurrent/TransactionCoordinator.java | 552 +++++++++-------- .../TransactionCoordinatorFactory.java | 12 +- .../sink/concurrent/TransactionManager.java | 71 ++- .../sink/concurrent/TransactionMode.java | 37 ++ .../sink/concurrent/TransactionState.java | 37 +- .../pixels/sink/config/CommandLineConfig.java | 16 +- .../pixels/sink/config/ConfigKey.java | 39 ++ .../pixels/sink/config/ConfigLoader.java | 78 +++ .../pixels/sink/config/PixelsSinkConfig.java | 169 ++--- .../sink/config/PixelsSinkConstants.java | 7 +- .../sink/config/PixelsSinkDefaultConfig.java | 8 +- .../pixels/sink/config/TransactionConfig.java | 6 +- .../sink/config/factory/KafkaPropFactory.java | 3 +- .../factory/KafkaPropFactorySelector.java | 12 +- .../factory/PixelsSinkConfigFactory.java | 25 +- .../factory/RowRecordKafkaPropFactory.java | 27 +- .../factory/TransactionKafkaPropFactory.java | 8 +- .../sink/deserializer/DeserializerUtil.java | 91 ++- .../RowChangeEventAvroDeserializer.java | 62 +- .../RowChangeEventJsonDeserializer.java | 136 ++-- .../RowChangeEventStructDeserializer.java | 132 ++++ .../sink/deserializer/RowDataParser.java | 346 +++++++++-- .../deserializer/RowRecordDeserializer.java | 62 -- .../sink/deserializer/SchemaDeserializer.java | 90 ++- .../TransactionAvroMessageDeserializer.java | 49 +- .../TransactionJsonMessageDeserializer.java | 22 +- .../TransactionStructMessageDeserializer.java | 78 +++ .../pixels/sink/event/RowChangeEvent.java | 232 ++++--- .../event/TableEnginePipelineManager.java | 57 ++ .../sink/event/TableEventEngineProvider.java | 90 +++ .../pixels/sink/event/TableEventProvider.java | 32 + .../sink/event/TablePipelineManager.java | 40 ++ .../event/TransactionEventEngineProvider.java | 58 ++ .../event/TransactionEventKafkaProvider.java | 107 ++++ .../sink/event/TransactionEventProvider.java | 34 + .../pixels/sink/exception/SinkException.java | 38 ++ .../pixels/sink/metadata/SchemaCache.java | 46 -- .../pixels/sink/metadata/TableMetadata.java | 57 +- .../sink/metadata/TableMetadataKey.java | 50 -- .../sink/metadata/TableMetadataRegistry.java | 112 ++-- .../sink/monitor/TransactionMonitor.java | 95 --- .../pixels/sink/processor/MainProcessor.java | 31 + .../{monitor => processor}/MetricsFacade.java | 99 ++- .../MonitorThreadManager.java | 38 +- .../sink/processor/SinkEngineProcessor.java | 84 +++ .../SinkKafkaProcessor.java} | 35 +- .../StoppableProcessor.java} | 8 +- .../TableMonitor.java} | 125 ++-- .../pixels/sink/processor/TableProcessor.java | 89 +++ .../TopicProcessor.java} | 162 +++-- .../sink/processor/TransactionProcessor.java | 79 +++ .../pixelsdb/pixels/sink/sink/CsvWriter.java | 113 ++-- .../pixels/sink/sink/PixelsSinkMode.java | 12 +- .../pixels/sink/sink/PixelsSinkWriter.java | 9 +- .../sink/sink/PixelsSinkWriterFactory.java | 9 +- .../pixels/sink/sink/RetinaWriter.java | 276 +++++---- .../pixels/sink/sink/TableCrossTxWriter.java | 189 ++++++ .../pixels/sink/sink/TableSingleTxWriter.java | 137 +++++ .../pixels/sink/sink/TableWriter.java | 173 ++++++ .../pixelsdb/pixels/sink/util/DateUtil.java | 89 +++ .../pixels/sink/util/LatencySimulator.java | 18 +- src/main/resources/log4j2.properties | 51 +- src/main/resources/logging.properties | 3 + src/main/resources/pixels-sink.aws.properties | 74 +++ .../resources/pixels-sink.local.properties | 64 +- src/main/resources/pixels-sink.properties | 5 +- .../pixels/sink/DebeziumEngineTest.java | 98 +++ .../io/pixelsdb/pixels/sink/TestUtils.java | 9 +- .../TransactionCoordinatorTest.java | 95 ++- .../concurrent/TransactionServiceTest.java | 61 +- .../sink/consumer/AvroConsumerTest.java | 104 ++-- .../sink/deserializer/RowBatchTest.java | 43 ++ .../RowChangeEventDeserializerTest.java | 12 +- .../sink/deserializer/RowDataParserTest.java | 55 +- .../deserializer/SchemaDeserializerTest.java | 30 +- .../sink/metadata/TestIndexService.java | 168 +++++ .../pixels/sink/sink/TestRetinaWriter.java | 582 ++++++++++++++++++ .../pixelsdb/pixels/sink/sink/TpcHTest.java | 339 ++++++++++ src/test/resources/log4j2.properties | 14 +- 94 files changed, 6175 insertions(+), 1715 deletions(-) create mode 100644 conf/jvm.conf create mode 100644 develop/docker-monitor-compose.yml create mode 100644 develop/example/sql/pg_replica.sql create mode 100644 develop/images/prometheus/prometheus_local.yml create mode 100755 pixels-sink create mode 100644 src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/deserializer/RowRecordDeserializer.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/metadata/SchemaCache.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataKey.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/monitor/TransactionMonitor.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java rename src/main/java/io/pixelsdb/pixels/sink/{monitor => processor}/MetricsFacade.java (75%) rename src/main/java/io/pixelsdb/pixels/sink/{monitor => processor}/MonitorThreadManager.java (72%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java rename src/main/java/io/pixelsdb/pixels/sink/{monitor/SinkMonitor.java => processor/SinkKafkaProcessor.java} (67%) rename src/main/java/io/pixelsdb/pixels/sink/{monitor/StoppableMonitor.java => processor/StoppableProcessor.java} (84%) rename src/main/java/io/pixelsdb/pixels/sink/{consumer/TableConsumerTask.java => processor/TableMonitor.java} (51%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java rename src/main/java/io/pixelsdb/pixels/sink/{monitor/TopicMonitor.java => processor/TopicProcessor.java} (69%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java create mode 100644 src/main/resources/logging.properties create mode 100644 src/main/resources/pixels-sink.aws.properties create mode 100644 src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/metadata/TestIndexService.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java diff --git a/conf/jvm.conf b/conf/jvm.conf new file mode 100644 index 0000000..011f5d4 --- /dev/null +++ b/conf/jvm.conf @@ -0,0 +1,24 @@ +-server +-XX:+AlwaysPreTouch +-Dfile.encoding=UTF-8 +-Duser.timezone=UTC + +-Xms8g +-Xmx16g + +-XX:+UseG1GC +-XX:MaxGCPauseMillis=200 +-XX:InitiatingHeapOccupancyPercent=35 +-XX:+ParallelRefProcEnabled +-XX:+UnlockExperimentalVMOptions +-XX:+TrustFinalNonStaticFields +-XX:+DisableExplicitGC + +-Xss512k + + +-XX:+HeapDumpOnOutOfMemoryError +-XX:HeapDumpPath=/var/log/java/java_heapdump.hprof +-XX:+ExitOnOutOfMemoryError + +-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:10086 diff --git a/develop/config/register-postgres.json.template b/develop/config/register-postgres.json.template index 283ed65..0b3884b 100644 --- a/develop/config/register-postgres.json.template +++ b/develop/config/register-postgres.json.template @@ -11,21 +11,15 @@ "database.dbname" : "pixels_realtime_crud", "schema.include.list": "public", "database.server.id": "1", - "topic.prefix": "oltp_server", + "topic.prefix": "postgresql.oltp_server", "transforms": "topicRouting", "transforms.topicRouting.type": "org.apache.kafka.connect.transforms.RegexRouter", - "transforms.topicRouting.regex": "oltp_server\\.public\\.(.*)", - "transforms.topicRouting.replacement": "oltp_server.pixels_realtime_crud.$1", + "transforms.topicRouting.regex": "postgresql.oltp_server\\.public\\.(.*)", + "transforms.topicRouting.replacement": "postgresql.oltp_server.pixels_realtime_crud.$1", - "key.converter": "io.apicurio.registry.utils.converter.AvroConverter", - "value.converter": "io.apicurio.registry.utils.converter.AvroConverter", - "key.converter.apicurio.registry.url": "http://apicurio:8080/apis/registry/v2", - "key.converter.apicurio.registry.auto-register": "true", - "key.converter.apicurio.registry.find-latest": "true", - "value.converter.apicurio.registry.url": "http://apicurio:8080/apis/registry/v2", - "value.converter.apicurio.registry.auto-register": "true", - "value.converter.apicurio.registry.find-latest": "true", - "schema.name.adjustment.mode": "avro" + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "plugin.name": "pgoutput" } } diff --git a/develop/docker-compose.yml b/develop/docker-compose.yml index 08621ff..d919b48 100644 --- a/develop/docker-compose.yml +++ b/develop/docker-compose.yml @@ -31,7 +31,7 @@ services: - pixels_network postgres: - image: quay.io/debezium/postgres:17 # This image install plugin: postgres-decoderbufs and configure wal_level = logical + image: quay.io/debezium/postgres:16 # This image install plugin: postgres-decoderbufs and configure wal_level = logical container_name: pixels_postgres_source_db environment: POSTGRES_DB: pixels_realtime_crud @@ -75,7 +75,8 @@ services: - pixels_network pixels-sink: - image: pixels-sink:0.2.0-SNAPSHOT + image: hello-world:latest + # image: pixels-sink:0.2.0-SNAPSHOT container_name: pixels-sink volumes: - ./data:/app/data @@ -101,7 +102,7 @@ services: - pixels_network pg_debezium: - image: debezium/connect:2.7.3.Final + image: debezium/connect:3.0.0.Final ports: - "8084:8083" depends_on: diff --git a/develop/docker-monitor-compose.yml b/develop/docker-monitor-compose.yml new file mode 100644 index 0000000..180be1b --- /dev/null +++ b/develop/docker-monitor-compose.yml @@ -0,0 +1,40 @@ +services: + # monitor + prometheus: + image: prom/prometheus:v3.2.1 + container_name: pixels-prometheus + ports: + - "9090:9090" + volumes: + - ./images/prometheus/prometheus_local.yml:/etc/prometheus/prometheus.yml + networks: + - pixels_monitor_network + extra_hosts: + - "host.docker.internal:host-gateway" + + grafana: + image: grafana/grafana:10.1.5 + container_name: pixels-grafana + ports: + - "3000:3000" + volumes: + - grafana-data:/var/lib/grafana + - ./images/grafana-provisioning:/etc/grafana/provisioning + environment: + - GF_SECURITY_ADMIN_USER=admin + - GF_SECURITY_ADMIN_PASSWORD=admin + - GF_AUTH_ANONYMOUS_ENABLED=true + - GF_AUTH_ANONYMOUS_ORG_ROLE=Admin + - GF_DEFAULT_INSTANCE_THEME=light + networks: + - pixels_monitor_network + depends_on: + - prometheus +volumes: + grafana-data: + + +networks: + pixels_monitor_network: + name: pixels_crud_network + driver: bridge diff --git a/develop/example/sql/dss.ri b/develop/example/sql/dss.ri index fb4c002..61b4382 100644 --- a/develop/example/sql/dss.ri +++ b/develop/example/sql/dss.ri @@ -1,100 +1,97 @@ -- Sccsid: @(#)dss.ri 2.1.8.1 --- TPCD Benchmark Version 8.0 +-- TPCH Benchmark Version 8.0 -CONNECT TO TPCD; +--CONNECT TO TPCH; ---ALTER TABLE TPCD.REGION DROP PRIMARY KEY; ---ALTER TABLE TPCD.NATION DROP PRIMARY KEY; ---ALTER TABLE TPCD.PART DROP PRIMARY KEY; ---ALTER TABLE TPCD.SUPPLIER DROP PRIMARY KEY; ---ALTER TABLE TPCD.PARTSUPP DROP PRIMARY KEY; ---ALTER TABLE TPCD.ORDERS DROP PRIMARY KEY; ---ALTER TABLE TPCD.LINEITEM DROP PRIMARY KEY; ---ALTER TABLE TPCD.CUSTOMER DROP PRIMARY KEY; +--ALTER TABLE REGION DROP PRIMARY KEY; +--ALTER TABLE NATION DROP PRIMARY KEY; +--ALTER TABLE PART DROP PRIMARY KEY; +--ALTER TABLE SUPPLIER DROP PRIMARY KEY; +--ALTER TABLE PARTSUPP DROP PRIMARY KEY; +--ALTER TABLE ORDERS DROP PRIMARY KEY; +--ALTER TABLE LINEITEM DROP PRIMARY KEY; +--ALTER TABLE CUSTOMER DROP PRIMARY KEY; -- For table REGION -ALTER TABLE TPCD.REGION +ALTER TABLE REGION ADD PRIMARY KEY (R_REGIONKEY); -- For table NATION -ALTER TABLE TPCD.NATION +ALTER TABLE NATION ADD PRIMARY KEY (N_NATIONKEY); -ALTER TABLE TPCD.NATION -ADD FOREIGN KEY NATION_FK1 (N_REGIONKEY) references TPCD.REGION; +ALTER TABLE NATION +ADD FOREIGN KEY (N_REGIONKEY) references REGION; -COMMIT WORK; +--COMMIT WORK; -- For table PART -ALTER TABLE TPCD.PART +ALTER TABLE PART ADD PRIMARY KEY (P_PARTKEY); -COMMIT WORK; +--COMMIT WORK; -- For table SUPPLIER -ALTER TABLE TPCD.SUPPLIER +ALTER TABLE SUPPLIER ADD PRIMARY KEY (S_SUPPKEY); -ALTER TABLE TPCD.SUPPLIER -ADD FOREIGN KEY SUPPLIER_FK1 (S_NATIONKEY) references TPCD.NATION; +ALTER TABLE SUPPLIER +ADD FOREIGN KEY (S_NATIONKEY) references NATION; -COMMIT WORK; +--COMMIT WORK; -- For table PARTSUPP -ALTER TABLE TPCD.PARTSUPP +ALTER TABLE PARTSUPP ADD PRIMARY KEY (PS_PARTKEY,PS_SUPPKEY); -COMMIT WORK; +--COMMIT WORK; -- For table CUSTOMER -ALTER TABLE TPCD.CUSTOMER +ALTER TABLE CUSTOMER ADD PRIMARY KEY (C_CUSTKEY); -ALTER TABLE TPCD.CUSTOMER -ADD FOREIGN KEY CUSTOMER_FK1 (C_NATIONKEY) references TPCD.NATION; +ALTER TABLE CUSTOMER +ADD FOREIGN KEY (C_NATIONKEY) references NATION; -COMMIT WORK; +--COMMIT WORK; -- For table LINEITEM -ALTER TABLE TPCD.LINEITEM +ALTER TABLE LINEITEM ADD PRIMARY KEY (L_ORDERKEY,L_LINENUMBER); -COMMIT WORK; +--COMMIT WORK; -- For table ORDERS -ALTER TABLE TPCD.ORDERS +ALTER TABLE ORDERS ADD PRIMARY KEY (O_ORDERKEY); -COMMIT WORK; +--COMMIT WORK; -- For table PARTSUPP -ALTER TABLE TPCD.PARTSUPP -ADD FOREIGN KEY PARTSUPP_FK1 (PS_SUPPKEY) references TPCD.SUPPLIER; +ALTER TABLE PARTSUPP +ADD FOREIGN KEY (PS_SUPPKEY) references SUPPLIER; -COMMIT WORK; +--COMMIT WORK; -ALTER TABLE TPCD.PARTSUPP -ADD FOREIGN KEY PARTSUPP_FK2 (PS_PARTKEY) references TPCD.PART; +ALTER TABLE PARTSUPP +ADD FOREIGN KEY (PS_PARTKEY) references PART; -COMMIT WORK; +--COMMIT WORK; -- For table ORDERS -ALTER TABLE TPCD.ORDERS -ADD FOREIGN KEY ORDERS_FK1 (O_CUSTKEY) references TPCD.CUSTOMER; +ALTER TABLE ORDERS +ADD FOREIGN KEY (O_CUSTKEY) references CUSTOMER; -COMMIT WORK; +--COMMIT WORK; -- For table LINEITEM -ALTER TABLE TPCD.LINEITEM -ADD FOREIGN KEY LINEITEM_FK1 (L_ORDERKEY) references TPCD.ORDERS; +ALTER TABLE LINEITEM +ADD FOREIGN KEY (L_ORDERKEY) references ORDERS; -COMMIT WORK; - -ALTER TABLE TPCD.LINEITEM -ADD FOREIGN KEY LINEITEM_FK2 (L_PARTKEY,L_SUPPKEY) references - TPCD.PARTSUPP; - -COMMIT WORK; +--COMMIT WORK; +ALTER TABLE LINEITEM +ADD FOREIGN KEY (L_PARTKEY,L_SUPPKEY) references PARTSUPP; +--COMMIT WORK; \ No newline at end of file diff --git a/develop/example/sql/pg_replica.sql b/develop/example/sql/pg_replica.sql new file mode 100644 index 0000000..96eaf20 --- /dev/null +++ b/develop/example/sql/pg_replica.sql @@ -0,0 +1,15 @@ +ALTER TABLE customer REPLICA IDENTITY USING INDEX customer_pkey; + +ALTER TABLE lineitem REPLICA IDENTITY USING INDEX lineitem_pkey; + +ALTER TABLE nation REPLICA IDENTITY USING INDEX nation_pkey; + +ALTER TABLE orders REPLICA IDENTITY USING INDEX orders_pkey; + +ALTER TABLE part REPLICA IDENTITY USING INDEX part_pkey; + +ALTER TABLE partsupp REPLICA IDENTITY USING INDEX partsupp_pkey; + +ALTER TABLE region REPLICA IDENTITY USING INDEX region_pkey; + +ALTER TABLE supplier REPLICA IDENTITY USING INDEX supplier_pkey; diff --git a/develop/images/grafana-provisioning/dashboards/sink-server.json b/develop/images/grafana-provisioning/dashboards/sink-server.json index 1e622ca..69729fc 100644 --- a/develop/images/grafana-provisioning/dashboards/sink-server.json +++ b/develop/images/grafana-provisioning/dashboards/sink-server.json @@ -21,17 +21,332 @@ "editable": true, "fiscalYearStartMonth": 0, "graphTooltip": 1, - "id": 1, "links": [], "liveNow": false, "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 9, + "x": 0, + "y": 0 + }, + "id": 27, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "editorMode": "code", + "expr": "jvm_memory_bytes_used{area=\"heap\"}", + "instant": false, + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Panel Title", + "type": "timeseries" + }, { "collapsed": false, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 0 + "y": 8 + }, + "id": 26, + "panels": [], + "title": "JVM", + "type": "row" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, + "id": 25, + "panels": [], + "title": "Debezium", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 11, + "w": 5, + "x": 0, + "y": 10 + }, + "id": 23, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(debezium_event_total[1m])", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "__auto", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Panel Title", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 11, + "w": 4, + "x": 5, + "y": 10 + }, + "id": 24, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "PBFA97CFB590B2093" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(row_event_total[$__rate_interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "__auto", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Panel Title", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 21 }, "id": 12, "panels": [], @@ -103,7 +418,7 @@ "h": 5, "w": 5, "x": 0, - "y": 1 + "y": 22 }, "id": 9, "options": { @@ -202,7 +517,7 @@ "h": 5, "w": 2, "x": 5, - "y": 1 + "y": 22 }, "id": 10, "options": { @@ -266,7 +581,7 @@ "h": 5, "w": 3, "x": 7, - "y": 1 + "y": 22 }, "id": 11, "options": { @@ -365,7 +680,7 @@ "h": 9, "w": 14, "x": 10, - "y": 1 + "y": 22 }, "id": 4, "options": { @@ -428,7 +743,7 @@ "h": 4, "w": 6, "x": 0, - "y": 6 + "y": 27 }, "id": 8, "options": { @@ -531,7 +846,7 @@ "h": 4, "w": 4, "x": 6, - "y": 6 + "y": 27 }, "id": 2, "options": { @@ -633,7 +948,7 @@ "h": 7, "w": 2, "x": 0, - "y": 10 + "y": 31 }, "id": 18, "options": { @@ -701,7 +1016,7 @@ "h": 7, "w": 4, "x": 2, - "y": 10 + "y": 31 }, "id": 20, "options": { @@ -770,7 +1085,7 @@ "h": 7, "w": 13, "x": 6, - "y": 10 + "y": 31 }, "id": 21, "options": { @@ -837,7 +1152,7 @@ "h": 7, "w": 5, "x": 19, - "y": 10 + "y": 31 }, "id": 6, "options": { @@ -877,7 +1192,7 @@ "h": 1, "w": 24, "x": 0, - "y": 17 + "y": 38 }, "id": 13, "panels": [], @@ -946,7 +1261,7 @@ "h": 9, "w": 4, "x": 0, - "y": 18 + "y": 39 }, "id": 14, "options": { @@ -1044,7 +1359,7 @@ "h": 9, "w": 4, "x": 4, - "y": 18 + "y": 39 }, "id": 15, "options": { @@ -1142,7 +1457,7 @@ "h": 9, "w": 4, "x": 8, - "y": 18 + "y": 39 }, "id": 16, "options": { @@ -1240,7 +1555,7 @@ "h": 9, "w": 6, "x": 12, - "y": 18 + "y": 39 }, "id": 17, "options": { @@ -1372,7 +1687,7 @@ "h": 9, "w": 6, "x": 18, - "y": 18 + "y": 39 }, "id": 22, "options": { @@ -1448,7 +1763,7 @@ "h": 1, "w": 24, "x": 0, - "y": 27 + "y": 48 }, "id": 19, "panels": [], @@ -1456,7 +1771,7 @@ "type": "row" } ], - "refresh": "auto", + "refresh": "5s", "schemaVersion": 38, "style": "light", "tags": [], diff --git a/develop/images/prometheus/prometheus_local.yml b/develop/images/prometheus/prometheus_local.yml new file mode 100644 index 0000000..340e1a6 --- /dev/null +++ b/develop/images/prometheus/prometheus_local.yml @@ -0,0 +1,9 @@ +global: + scrape_interval: 1s + +scrape_configs: + - job_name: 'pixels-sink' + metrics_path: /metrics + static_configs: + - targets: [ 'host.docker.internal:9464' ] + diff --git a/develop/scripts/install.sh b/develop/scripts/install.sh index 2906792..b741662 100755 --- a/develop/scripts/install.sh +++ b/develop/scripts/install.sh @@ -110,6 +110,7 @@ try_command curl -f -X POST -H "Content-Type: application/json" -d @${CONFIG_DIR check_fatal_exit "Register PostgreSQL Source Connector Fail" if [[ x${enable_tpch} == x"on" && x${load_postgres} == x"on" ]]; then docker exec pixels_postgres_source_db sh -c " psql -Upixels -d pixels_realtime_crud < /example/sql/dss.ddl" + docker exec pixels_postgres_source_db sh -c " psql -Upixels -d pixels_realtime_crud < /example/sql/dss.ri" docker exec pixels_postgres_source_db sh -c " psql -Upixels -d pixels_realtime_crud < /load.sql" fi fi diff --git a/pixels-sink b/pixels-sink new file mode 100755 index 0000000..754f061 --- /dev/null +++ b/pixels-sink @@ -0,0 +1,45 @@ +#!/usr/bin/env bash +set -euo pipefail + +# Resolve the absolute path of this script +SOURCE_PATH=$(readlink -f "$BASH_SOURCE") 2>/dev/null +SINK_DIR=$(dirname "$SOURCE_PATH") + +# Environment variable +export PIXELS_HOME="/home/ubuntu/opt/pixels" + +# Application properties file +PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" + +# JVM config file +JVM_CONFIG_FILE="${SINK_DIR}/conf/jvm.conf" + +if [[ ! -f "$JVM_CONFIG_FILE" ]]; then + echo "JVM config file not found: $JVM_CONFIG_FILE" + exit 1 +fi + +# Read JVM options (ignore comments and empty lines) +JVM_OPTS=$(grep -v '^\s*#' "$JVM_CONFIG_FILE" | grep -v '^\s*$' | xargs) + +# Main class (for reference, though fat-jar specifies it in MANIFEST) +MAIN_CLASS="io.pixelsdb.pixels.sink.PixelsSinkApp" + +# Application arguments +APP_ARGS="-c $PROPERTIES_FILE" + +# Path to the fat jar +APP_JAR="$SINK_DIR/target/pixels-sink-0.2.0-SNAPSHOT-full.jar" + +if [[ ! -f "$APP_JAR" ]]; then + echo "Application jar not found: $APP_JAR" + exit 1 +fi + +echo "Starting PixelsSinkApp..." +echo "PIXELS_HOME = $PIXELS_HOME" +echo "JVM_OPTS = $JVM_OPTS" +echo "APP_JAR = $APP_JAR" +echo "APP_ARGS = $APP_ARGS" + +exec java $JVM_OPTS -jar "$APP_JAR" $APP_ARGS diff --git a/pom.xml b/pom.xml index e9896e7..5e46b4f 100644 --- a/pom.xml +++ b/pom.xml @@ -37,7 +37,7 @@ 3.8.0 5.8 1.18.36 - 3.0.7.Final + 3.2.3.Final 1.4.13 @@ -52,13 +52,19 @@ pixels-core true + + io.pixelsdb + pixels-retina + true + test + + com.alibaba fastjson true - org.apache.logging.log4j log4j-core @@ -111,6 +117,11 @@ kafka-clients ${dep.kafka.version} + + org.apache.kafka + connect-api + ${dep.kafka.version} + com.opencsv @@ -142,7 +153,11 @@ debezium-sink ${dep.debezium.version} - + + io.debezium + debezium-connector-postgres + ${dep.debezium.version} + junit @@ -189,6 +204,11 @@ simpleclient 0.16.0 + + io.prometheus + simpleclient_hotspot + 0.16.0 + io.prometheus @@ -257,43 +277,24 @@ develop/images/pixels-sink/Dockerfile - - com.github.os72 - protoc-jar-maven-plugin - 3.3.0.1 - - - generate-sources - - run - - - - src/main/proto - - - - java - src/main/java - - - grpc-java - src/main/java - io.grpc:protoc-gen-grpc-java:1.53.0 - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - 16 - 16 - - + + org.apache.maven.plugins + maven-compiler-plugin + + 17 + 17 + + + org.projectlombok + lombok + ${dep.lombok.version} + + + + -parameters + + + \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java new file mode 100644 index 0000000..955d289 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java @@ -0,0 +1,142 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink; + + +import io.debezium.engine.DebeziumEngine; +import io.debezium.engine.RecordChangeEvent; +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.TableEnginePipelineManager; +import io.pixelsdb.pixels.sink.event.TablePipelineManager; +import io.pixelsdb.pixels.sink.event.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.processor.StoppableProcessor; +import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * @package: io.pixelsdb.pixels.sink + * @className: PixelsDebeziumConsumer + * @author: AntiO2 + * @date: 2025/9/25 12:51 + */ +public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer>, StoppableProcessor +{ + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + private final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); + private final String checkTransactionTopic; + private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; + private final TableEnginePipelineManager tableEnginePipelineManager = new TableEnginePipelineManager(); + private final TransactionProcessor processor = new TransactionProcessor(transactionEventProvider); + private final Thread processorThread; + private final Thread adapterThread; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + + public PixelsDebeziumConsumer() + { + this.checkTransactionTopic = pixelsSinkConfig.getDebeziumTopicPrefix() + ".transaction"; + adapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); + adapterThread.start(); + processorThread = new Thread(processor, "debezium-processor"); + processorThread.start(); + } + + + public void handleBatch(List> event, + DebeziumEngine.RecordCommitter> committer) throws InterruptedException { + for(RecordChangeEvent record:event) + { + try + { + SourceRecord sourceRecord = record.record(); + if(sourceRecord == null) + { + continue; + } + + metricsFacade.recordDebeziumEvent(); + if(isTransactionEvent(sourceRecord)) + { + metricsFacade.recordTransaction(); + // handleTransactionSourceRecord(sourceRecord); + } else { + metricsFacade.recordRowEvent(); + // handleRowChangeSourceRecord(sourceRecord); + } + } + finally + { + committer.markProcessed(record); + } + + } + committer.markBatchFinished(); + } + + private void handleTransactionSourceRecord(SourceRecord sourceRecord) throws InterruptedException + { + rawTransactionQueue.put(sourceRecord); + } + + private void handleRowChangeSourceRecord(SourceRecord sourceRecord) + { + Struct value = (Struct) sourceRecord.value(); + Struct source = (Struct) value.get("source"); + String schemaName = source.get("db").toString(); + String tableName = source.get("table").toString(); + SchemaTableName schemaTableName = new SchemaTableName(schemaName, tableName); + tableEnginePipelineManager.routeRecord(schemaTableName, sourceRecord); + } + + private void processTransactionSourceRecord() { + while (true) { + try { + SourceRecord sourceRecord = rawTransactionQueue.take(); + SinkProto.TransactionMetadata tx = transactionEventProvider.convert(sourceRecord); + if (tx != null) { + transactionEventProvider.getEventQueue().put(tx); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + private boolean isTransactionEvent(SourceRecord sourceRecord) + { + return checkTransactionTopic.equals(sourceRecord.topic()); + } + + @Override + public void stopProcessor() + { + adapterThread.interrupt(); + processor.stopProcessor(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 92f3632..3367e59 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -18,9 +18,11 @@ import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; import io.pixelsdb.pixels.sink.config.CommandLineConfig; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; -import io.pixelsdb.pixels.sink.monitor.SinkMonitor; +import io.pixelsdb.pixels.sink.processor.*; +import io.prometheus.client.hotspot.DefaultExports; +import io.prometheus.client.exporter.HTTPServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,24 +31,59 @@ /** * Run PixelsSink as a server */ -public class PixelsSinkApp { +public class PixelsSinkApp +{ private static final Logger LOGGER = LoggerFactory.getLogger(PixelsSinkApp.class); - private static SinkMonitor sinkMonitor = new SinkMonitor(); + private static MainProcessor mainProcessor; + private static HTTPServer prometheusHttpServer; - public static void main(String[] args) throws IOException { - Runtime.getRuntime().addShutdownHook(new Thread(() -> { - sinkMonitor.stopMonitor(); + + public static void main(String[] args) throws IOException + { + Runtime.getRuntime().addShutdownHook(new Thread(() -> + { + mainProcessor.stopProcessor(); TransactionCoordinatorFactory.reset(); LOGGER.info("Pixels Sink Server shutdown complete"); + if(prometheusHttpServer != null) + { + prometheusHttpServer.close(); + } + })); init(args); - sinkMonitor.startSinkMonitor(); + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + if(config.getDataSource().equals("kafka")) + { + mainProcessor = new SinkKafkaProcessor(); + } else if(config.getDataSource().equals("engine")) + { + mainProcessor = new SinkEngineProcessor(); + } else + { + throw new IllegalStateException("Unsupported data source type: " + config.getDataSource()); + } + + try + { + if (config.isMonitorEnabled()) + { + DefaultExports.initialize(); + prometheusHttpServer = new HTTPServer(config.getMonitorPort()); + } + } catch (IOException e) + { + throw new RuntimeException(e); + } + + mainProcessor.start(); } - private static void init(String[] args) throws IOException { + private static void init(String[] args) throws IOException + { CommandLineConfig cmdLineConfig = new CommandLineConfig(args); PixelsSinkConfigFactory.initialize(cmdLineConfig.getConfigPath()); - MetricsFacade.initialize(); + MetricsFacade.getInstance(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java index f7bdd71..ca642e2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java @@ -19,27 +19,45 @@ import io.pixelsdb.pixels.common.sink.SinkProvider; import io.pixelsdb.pixels.common.utils.ConfigFactory; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; -import io.pixelsdb.pixels.sink.monitor.SinkMonitor; +import io.pixelsdb.pixels.sink.processor.MainProcessor; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.processor.SinkEngineProcessor; +import io.pixelsdb.pixels.sink.processor.SinkKafkaProcessor; -public class PixelsSinkProvider implements SinkProvider { - private SinkMonitor sinkMonitor; +public class PixelsSinkProvider implements SinkProvider +{ + private MainProcessor mainProcessor; - public void start(ConfigFactory config) { + public void start(ConfigFactory config) + { PixelsSinkConfigFactory.initialize(config); - MetricsFacade.initialize(); - sinkMonitor = new SinkMonitor(); - sinkMonitor.startSinkMonitor(); + MetricsFacade.getInstance(); + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + String dataSource = pixelsSinkConfig.getDataSource(); + if(dataSource.equals("kafka")) + { + mainProcessor = new SinkKafkaProcessor(); + } else if(dataSource.equals("engine")) + { + mainProcessor = new SinkEngineProcessor(); + } else + { + throw new IllegalStateException("Unsupported data source type: " + dataSource); + } + mainProcessor.start(); } @Override - public void shutdown() { - sinkMonitor.stopMonitor(); + public void shutdown() + { + mainProcessor.stopProcessor(); } @Override - public boolean isRunning() { - return sinkMonitor.isRunning(); + public boolean isRunning() + { + return mainProcessor.isRunning(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java index b30f350..ddd84e3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java @@ -19,43 +19,57 @@ import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Map; +import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -class SinkContext { +public class SinkContext +{ private static final Logger LOGGER = LoggerFactory.getLogger(SinkContext.class); final ReentrantLock lock = new ReentrantLock(); - final Condition cond = lock.newCondition(); + final Condition cond = lock.newCondition(); // this cond is wait for pixels tx + + final ReentrantLock tableCounterLock = new ReentrantLock(); + final Condition tableCounterCond = tableCounterLock.newCondition(); + final String sourceTxId; final Map tableCursors = new ConcurrentHashMap<>(); final Map tableCounters = new ConcurrentHashMap<>(); final AtomicInteger pendingEvents = new AtomicInteger(0); final CompletableFuture completionFuture = new CompletableFuture<>(); - TransContext pixelsTransCtx; - volatile boolean completed = false; + final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + volatile boolean completed = false; + Queue orphanEvent = new ConcurrentLinkedQueue<>(); + private TransContext pixelsTransCtx; - SinkContext(String sourceTxId) { + SinkContext(String sourceTxId) + { this.sourceTxId = sourceTxId; this.pixelsTransCtx = null; } - SinkContext(String sourceTxId, TransContext pixelsTransCtx) { + SinkContext(String sourceTxId, TransContext pixelsTransCtx) + { this.sourceTxId = sourceTxId; this.pixelsTransCtx = pixelsTransCtx; } - boolean isReadyForDispatch(String table, long collectionOrder) { + boolean isReadyForDispatch(String table, long collectionOrder) + { lock.lock(); boolean ready = tableCursors .computeIfAbsent(table, k -> 1L) >= collectionOrder; @@ -63,44 +77,150 @@ boolean isReadyForDispatch(String table, long collectionOrder) { return ready; } - void updateCursor(String table, long currentOrder) { + void updateCursor(String table, long currentOrder) + { tableCursors.compute(table, (k, v) -> (v == null) ? currentOrder + 1 : Math.max(v, currentOrder + 1)); } - void updateCounter(String table) { + void updateCounter(String table) + { + updateCounter(table, 1L); + } + + public void updateCounter(String table, long count) + { + tableCounterLock.lock(); tableCounters.compute(table, (k, v) -> - (v == null) ? 1 : v + 1); + (v == null) ? count : v + count); + tableCounterCond.signalAll(); + tableCounterLock.unlock(); } - Set getTrackedTables() { + public ReentrantLock getTableCounterLock() + { + return tableCounterLock; + } + + public Condition getTableCounterCond() + { + return tableCounterCond; + } + + Set getTrackedTables() + { return tableCursors.keySet(); } - boolean isCompleted(SinkProto.TransactionMetadata tx) { - for (SinkProto.DataCollection dataCollection : tx.getDataCollectionsList()) { + boolean isCompleted(SinkProto.TransactionMetadata tx) + { + for (SinkProto.DataCollection dataCollection : tx.getDataCollectionsList()) + { // Long targetEventCount = tableCursors.get(dataCollection.getDataCollection()); Long targetEventCount = tableCounters.get(dataCollection.getDataCollection()); long target = targetEventCount == null ? 0 : targetEventCount; LOGGER.debug("TX {}, Table {}, event count {}, tableCursors {}", tx.getId(), dataCollection.getDataCollection(), dataCollection.getEventCount(), target); - if (dataCollection.getEventCount() > target) { + if (dataCollection.getEventCount() > target) + { return false; } } return true; } - boolean isExpired() { + boolean isExpired() + { // TODO: expire timeout transaction return false; // return System.currentTimeMillis() - pixelsTransCtx.getTimestamp() > TX_TIMEOUT_MS; } - void markCompleted() { + void markCompleted() + { this.completed = true; } - void awaitCompletion() throws InterruptedException, ExecutionException { + void awaitCompletion() throws InterruptedException, ExecutionException + { completionFuture.get(); } + + public long getTimestamp() + { + if(pixelsTransCtx == null) + { + throw new RuntimeException("PixelsTransCtx is NULL"); + } + return pixelsTransCtx.getTimestamp(); + } + + public void bufferOrphanedEvent(RowChangeEvent event) + { + orphanEvent.add(event); + } + + public Queue getOrphanEvent() + { + return orphanEvent; + } + + public ReentrantLock getLock() + { + return lock; + } + + public Condition getCond() + { + return cond; + } + + public String getSourceTxId() + { + return sourceTxId; + } + + public Map getTableCursors() + { + return tableCursors; + } + + public Map getTableCounters() + { + return tableCounters; + } + + public AtomicInteger getPendingEvents() + { + return pendingEvents; + } + + public CompletableFuture getCompletionFuture() + { + return completionFuture; + } + + public TableMetadataRegistry getTableMetadataRegistry() + { + return tableMetadataRegistry; + } + + public TransContext getPixelsTransCtx() + { + return pixelsTransCtx; + } + + public void setPixelsTransCtx(TransContext pixelsTransCtx) + { + this.pixelsTransCtx = pixelsTransCtx; + } + + public boolean isCompleted() + { + return completed; + } + + public void setCompleted(boolean completed) + { + this.completed = completed; + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java index d5412a0..6a65182 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java @@ -20,50 +20,53 @@ import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; import io.pixelsdb.pixels.sink.sink.PixelsSinkWriterFactory; -import io.pixelsdb.pixels.sink.util.LatencySimulator; +import io.pixelsdb.pixels.sink.sink.TableCrossTxWriter; +import io.pixelsdb.pixels.sink.sink.TableSingleTxWriter; import io.prometheus.client.Summary; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Comparator; import java.util.List; +import java.util.Queue; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -public class TransactionCoordinator { - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionCoordinator.class); +public class TransactionCoordinator +{ public static final int INITIAL_CAPACITY = 11; - private final PixelsSinkWriter writer; - + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionCoordinator.class); + private static final int MAX_ACTIVE_TX = 1000; final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(); final ExecutorService dispatchExecutor = Executors.newCachedThreadPool(); - private final ExecutorService transactionExecutor = Executors.newCachedThreadPool(); - private final ConcurrentMap> orphanedEvents = new ConcurrentHashMap<>(); - private final ConcurrentMap> orderedBuffers = new ConcurrentHashMap<>(); - // private final BlockingQueue nonTxQueue = new LinkedBlockingQueue<>(); - private long TX_TIMEOUT_MS = PixelsSinkConfigFactory.getInstance().getTransactionTimeout(); + private final PixelsSinkWriter writer; + private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(1024); private final ScheduledExecutorService timeoutScheduler = Executors.newSingleThreadScheduledExecutor(); - private final TransactionManager transactionManager = TransactionManager.Instance(); private final TransService transService; - private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + // private final BlockingQueue nonTxQueue = new LinkedBlockingQueue<>(); + private long TX_TIMEOUT_MS = PixelsSinkConfigFactory.getInstance().getTransactionTimeout(); - - TransactionCoordinator() { - try { + TransactionCoordinator() + { + try + { this.writer = PixelsSinkWriterFactory.getWriter(); - } catch (IOException e) { + } catch (IOException e) + { throw new RuntimeException(e); } transService = TransService.Instance(); @@ -71,23 +74,29 @@ public class TransactionCoordinator { startTimeoutChecker(); } - public void processTransactionEvent(SinkProto.TransactionMetadata txMeta) { - if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) { + public void processTransactionEvent(SinkProto.TransactionMetadata txMeta) throws SinkException + { + if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) + { handleTxBegin(txMeta); - } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) { + } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) + { handleTxEnd(txMeta); metricsFacade.recordTransaction(); } } - public void processRowEvent(RowChangeEvent event) { - if (event == null) { + public void processRowEvent(RowChangeEvent event) throws SinkException + { + if (event == null) + { return; } metricsFacade.recordRowChange(event.getTable(), event.getOp()); event.startLatencyTimer(); - if (event.getTransaction() == null || event.getTransaction().getId().isEmpty()) { + if (event.getTransaction() == null || event.getTransaction().getId().isEmpty()) + { handleNonTxEvent(event); return; } @@ -99,235 +108,231 @@ public void processRowEvent(RowChangeEvent event) { long totalOrder = event.getTransaction().getTotalOrder(); LOGGER.debug("Receive event {} {}/{} {}/{} ", event.getOp().toString(), txId, totalOrder, table, collectionOrder); - SinkContext ctx = activeTxContexts.get(txId); - if (ctx == null) { - // async method - // try { - // ctx = startTrans(txId).get(); - // } catch (InterruptedException | ExecutionException e) { - // throw new RuntimeException(e); - // } - - // sync mode: we should wait for transaction message - bufferOrphanedEvent(event); - return; - } - ctx.lock.lock(); - try { - ctx.cond.signalAll(); - } finally { - ctx.lock.unlock(); - } - - OrderedEvent orderedEvent = new OrderedEvent(event, collectionOrder, totalOrder); -// if (ctx.isReadyForDispatch(table, collectionOrder)) { - if(true) { - LOGGER.debug("Immediately dispatch {} {}/{}", event.getTransaction().getId(), collectionOrder, totalOrder); - ctx.pendingEvents.incrementAndGet(); - dispatchImmediately(event, ctx); - // ctx.updateCursor(table, collectionOrder); - ctx.updateCounter(table); - checkPendingEvents(ctx, table); - } else { - bufferOrderedEvent(ctx, orderedEvent); + AtomicBoolean canWrite = new AtomicBoolean(false); + SinkContext ctx = activeTxContexts.compute(txId, (sourceTxId, sinkContext) -> + { + if (sinkContext == null) + { + SinkContext newSinkContext = new SinkContext(sourceTxId); + newSinkContext.bufferOrphanedEvent(event); + return newSinkContext; + } else + { + if (sinkContext.getPixelsTransCtx() == null) + { + sinkContext.bufferOrphanedEvent(event); + return sinkContext; + } + canWrite.set(true); + return sinkContext; + } + }); + if(canWrite.get()) + { + processRowChangeEvent(ctx, event); } } - private void handleTxBegin(SinkProto.TransactionMetadata txBegin) { + private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkException + { // startTrans(txBegin.getId()).get(); - startTransSync(txBegin.getId()); - } - - private void startTransSync(String sourceTxId) { - SinkContext ctx = activeTxContexts.computeIfAbsent(sourceTxId, k -> new SinkContext(sourceTxId)); - TransContext pixelsTransContext; - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - if (pixelsSinkConfig.isRpcEnable()) { - pixelsTransContext = transactionManager.getTransContext(); - } else { - LatencySimulator.smartDelay(); - pixelsTransContext = new TransContext(sourceTxId.hashCode(), System.currentTimeMillis(), false); - } - transLatencyTimer.close(); - ctx.pixelsTransCtx = pixelsTransContext; - List buffered = getBufferedEvents(sourceTxId); - if (buffered != null) { - buffered.forEach(be -> processBufferedEvent(ctx, be)); + try + { + startTransSync(txBegin.getId()); + } catch (SinkException e) + { + throw new SinkException("Failed to start trans", e); } - LOGGER.info("Begin Tx Sync: {}", sourceTxId); - } - @Deprecated - private Future startTrans(String sourceTxId) { - SinkContext ctx = activeTxContexts.computeIfAbsent(sourceTxId, k -> new SinkContext(sourceTxId)); - return transactionExecutor.submit(() -> { - try { - ctx.lock.lock(); - TransContext pixelsTransContext; - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - if (pixelsSinkConfig.isRpcEnable()) { - pixelsTransContext = transService.beginTrans(false); - } else { - LatencySimulator.smartDelay(); - pixelsTransContext = new TransContext(sourceTxId.hashCode(), System.currentTimeMillis(), false); - } - transLatencyTimer.close(); - activeTxContexts.get(sourceTxId).pixelsTransCtx = pixelsTransContext; - ctx.lock.unlock(); - List buffered = getBufferedEvents(sourceTxId); - if (buffered != null) { - buffered - // .stream() - // .sorted(Comparator.comparingLong(BufferedEvent::getTotalOrder)) - .forEach(be -> processBufferedEvent(ctx, be)); - } - } catch (TransException e) { - throw new RuntimeException(e); - } - LOGGER.info("Begin Tx: {}", sourceTxId); - return ctx; - }); } - private void handleTxEnd(SinkProto.TransactionMetadata txEnd) { - String txId = txEnd.getId(); - SinkContext ctx = activeTxContexts.get(txId); - transactionExecutor.submit(() -> { - LOGGER.info("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - if (ctx != null) { - try { - ctx.lock.lock(); - ctx.markCompleted(); - try { - while (!ctx.isCompleted(txEnd)) { - ctx.lock.lock(); - LOGGER.debug("TX End Get Lock {}", txId); - LOGGER.debug("Waiting for events in TX {}: {}", txId, - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - - ctx.cond.await(100, TimeUnit.MILLISECONDS); - } - } finally { - ctx.lock.unlock(); - } - - if (ctx.pendingEvents.get() > 0) { - LOGGER.info("Waiting for {} pending events in TX {}", - ctx.pendingEvents.get(), txId); - ctx.awaitCompletion(); - } - - flushRemainingEvents(ctx); - activeTxContexts.remove(txId); - LOGGER.info("Committed transaction: {}", txId); - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - if (pixelsSinkConfig.isRpcEnable()) { - transService.commitTrans(ctx.pixelsTransCtx.getTransId(), ctx.pixelsTransCtx.getTimestamp()); - } else { - LatencySimulator.smartDelay(); - } - transLatencyTimer.close(); - } catch (InterruptedException | ExecutionException | TransException e) { - // TODO(AntiO2) abort? - LOGGER.error("Failed to commit transaction {}", txId, e); + private void startTransSync(String sourceTxId) throws SinkException + { + activeTxContexts.compute( + sourceTxId, + (k, oldCtx) -> + { + if (oldCtx == null) + { + return new SinkContext(sourceTxId, transactionManager.getTransContext()); + } else + { + oldCtx.getLock().lock(); + try + { + oldCtx.setPixelsTransCtx(transactionManager.getTransContext()); + handleOrphanEvents(oldCtx); + oldCtx.getCond().signalAll(); + } catch (SinkException e) + { + throw new RuntimeException(e); + } finally + { + oldCtx.getLock().unlock(); } + return oldCtx; } } ); + LOGGER.info("Begin Tx Sync: {}", sourceTxId); } + private void handleOrphanEvents(SinkContext ctx) throws SinkException + { + Queue buffered = ctx.getOrphanEvent(); - private void bufferOrphanedEvent(RowChangeEvent event) { - orphanedEvents.computeIfAbsent(event.getTransaction().getId(), k -> new CopyOnWriteArrayList<>()).add(event); - // LOGGER.debug("Buffered orphan event for TX {}: {}/{}", txId, event.collectionOrder, event.totalOrder); + if (buffered != null) + { + for (RowChangeEvent event : buffered) + { + processRowChangeEvent(ctx, event); + } + } } - private List getBufferedEvents(String txId) { - return orphanedEvents.remove(txId); - } + private void handleTxEnd(SinkProto.TransactionMetadata txEnd) + { + String txId = txEnd.getId(); + SinkContext ctx = getSinkContext(txId); + + transactionExecutor.submit(() -> + { + processTxCommit(txEnd, txId, ctx); + } + ); + switch (pixelsSinkConfig.getTransactionMode()) + { - private void processBufferedEvent(SinkContext ctx, RowChangeEvent bufferedEvent) { - String table = bufferedEvent.getTable(); - dispatchImmediately(bufferedEvent, ctx); - -// long collectionOrder = bufferedEvent.collectionOrder; -// if (ctx.isReadyForDispatch(table, collectionOrder)) { -// dispatchImmediately(bufferedEvent.event, ctx); -// ctx.lock.lock(); -// ctx.updateCursor(table, collectionOrder); -// ctx.lock.unlock(); -// checkPendingEvents(ctx, table); -// } else { -// bufferOrderedEvent(ctx, new OrderedEvent( -// bufferedEvent.event, -// collectionOrder, -// bufferedEvent.totalOrder -// )); -// ctx.pendingEvents.incrementAndGet(); // track pending events -// } +// case BATCH -> +// { +// processTxCommit(txEnd, txId, ctx); +// } +// case RECORD -> +// { +// transactionExecutor.submit(() -> +// { +// processTxCommit(txEnd, txId, ctx); +// } +// ); +// } + } } - private void bufferOrderedEvent(SinkContext ctx, OrderedEvent event) { - String bufferKey = ctx.sourceTxId + "|" + event.getTable(); - LOGGER.info("Buffered out-of-order event: {} {}/{}. Pending Events: {}", - bufferKey, event.collectionOrder, event.totalOrder, ctx.pendingEvents.incrementAndGet()); - orderedBuffers.computeIfAbsent(bufferKey, k -> - new PriorityBlockingQueue<>(INITIAL_CAPACITY, Comparator.comparingLong(OrderedEvent::getCollectionOrder)) - ).offer(event); + private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, SinkContext ctx) + { + LOGGER.info("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), + txEnd.getDataCollectionsList().stream() + .map(dc -> dc.getDataCollection() + "=" + + ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + + "/" + dc.getEventCount()) + .collect(Collectors.joining(", "))); + if (ctx == null) + { + LOGGER.warn("Sink Context is null"); + return; + } + + try + { + try { + ctx.tableCounterLock.lock(); + while (!ctx.isCompleted(txEnd)) + { + LOGGER.debug("TX End Get Lock {}", txId); + LOGGER.debug("Waiting for events in TX {}: {}", txId, + txEnd.getDataCollectionsList().stream() + .map(dc -> dc.getDataCollection() + "=" + + ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + + "/" + dc.getEventCount()) + .collect(Collectors.joining(", "))); + ctx.tableCounterCond.await(); + } + } finally + { + ctx.tableCounterLock.unlock(); + } + + + activeTxContexts.remove(txId); + boolean res = true; + if (res) + { + LOGGER.info("Committed transaction: {}", txId); + Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); + transactionManager.commitTransAsync(ctx.getPixelsTransCtx()); + } else + { + LOGGER.info("Abort transaction: {}", txId); + Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); + CompletableFuture.runAsync(() -> + { + try + { + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId()); + } catch (TransException e) + { + throw new RuntimeException(e); + } + }).whenComplete((v, ex) -> + { + transLatencyTimer.close(); + if (ex != null) + { + LOGGER.error("Rollback failed", ex); + } + }); + } + } catch (InterruptedException e) + { + try + { + LOGGER.info("Catch Exception, Abort transaction: {}", txId); + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId()); + } catch (TransException ex) + { + LOGGER.error("Failed to abort transaction {}", txId); + ex.printStackTrace(); + LOGGER.error(ex.getMessage()); + throw new RuntimeException(ex); + } + LOGGER.error(e.getMessage()); + LOGGER.error("Failed to commit transaction {}", txId, e); + } } - private void checkPendingEvents(SinkContext ctx, String table) { - String bufferKey = ctx.sourceTxId + "|" + table; - PriorityBlockingQueue buffer = orderedBuffers.get(bufferKey); - if (buffer == null) return; - - while (!buffer.isEmpty()) { - OrderedEvent nextEvent = buffer.peek(); - if (ctx.isReadyForDispatch(table, nextEvent.collectionOrder)) { - LOGGER.debug("Ordered buffer dispatch {} {}/{}", bufferKey, nextEvent.collectionOrder, nextEvent.totalOrder); - dispatchImmediately(nextEvent.event, ctx); - buffer.poll(); - } else { - break; + private void processRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException + { + String table = event.getTable(); + event.setTimeStamp(ctx.getTimestamp()); + event.initIndexKey(); + switch (pixelsSinkConfig.getTransactionMode()) + { + case BATCH -> + { + TableCrossTxWriter.getTableWriter(table).write(event, ctx); + } + case TRANS -> + { + TableSingleTxWriter.getTableWriter(table).write(event, ctx); + } + case RECORD -> + { + dispatchImmediately(event, ctx); } } } - private void startDispatchWorker() { -// dispatchExecutor.execute(() -> { -// while (!Thread.currentThread().isInterrupted()) { -// try { -// RowChangeEvent event = nonTxQueue.poll(10, TimeUnit.MILLISECONDS); -// if (event != null) { -// dispatchImmediately(event, null); -// metricsFacade.recordTransaction(); -// continue; -// } -// -// activeTxContexts.values().forEach(ctx -> -// ctx.getTrackedTables().forEach(table -> -// checkPendingEvents(ctx, table) -// ) -// ); -// } catch (InterruptedException e) { -// Thread.currentThread().interrupt(); -// } -// } -// }); + public SinkContext getSinkContext(String txId) + { + return activeTxContexts.get(txId); } - protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) { - dispatchExecutor.execute(() -> { - try { + protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) + { + dispatchExecutor.execute(() -> + { + try + { LOGGER.debug("Dispatching [{}] {}.{} (Order: {}/{}) TX: {}", event.getOp().name(), event.getDb(), @@ -340,18 +345,22 @@ protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) { Summary.Timer writeLatencyTimer = metricsFacade.startWriteLatencyTimer(); boolean success = writer.write(event); writeLatencyTimer.close(); - if (success) { + if (success) + { metricsFacade.recordTotalLatency(event); metricsFacade.recordRowChange(event.getTable(), event.getOp()); event.endLatencyTimer(); - } else { + } else + { // TODO retry? } - - } finally { - if (ctx != null) { + } finally + { + if (ctx != null) + { ctx.updateCounter(event.getFullTableName()); - if (ctx.pendingEvents.decrementAndGet() == 0 && ctx.completed) { + if (ctx.pendingEvents.decrementAndGet() == 0 && ctx.completed) + { ctx.completionFuture.complete(null); } } @@ -359,13 +368,16 @@ protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) { }); } - private void startTimeoutChecker() { - timeoutScheduler.scheduleAtFixedRate(() -> { - activeTxContexts.entrySet().removeIf(entry -> { + private void startTimeoutChecker() + { + timeoutScheduler.scheduleAtFixedRate(() -> + { + activeTxContexts.entrySet().removeIf(entry -> + { SinkContext ctx = entry.getValue(); - if (ctx.isExpired()) { + if (ctx.isExpired()) + { LOGGER.warn("Transaction timeout: {}", entry.getKey()); - flushRemainingEvents(ctx); return true; } return false; @@ -373,75 +385,81 @@ private void startTimeoutChecker() { }, 10, 10, TimeUnit.SECONDS); } - private void flushRemainingEvents(SinkContext ctx) { - LOGGER.debug("Try Flush remaining events of {}", ctx.sourceTxId); - ctx.getTrackedTables().forEach(table -> { - String bufferKey = ctx.sourceTxId + "|" + table; - PriorityBlockingQueue buffer = orderedBuffers.remove(bufferKey); - if (buffer != null) { - LOGGER.warn("Flushing {} events for {}.{}", - buffer.size(), ctx.sourceTxId, table); - buffer.forEach(event -> { - LOGGER.debug("Processing event for {}:{}/{}", - ctx.sourceTxId, event.collectionOrder, event.totalOrder); - dispatchImmediately(event.event, ctx); - LOGGER.debug("End Event for {}:{}/{}", - ctx.sourceTxId, event.collectionOrder, event.totalOrder); - }); + private void handleNonTxEvent(RowChangeEvent event) throws SinkException + { + switch (pixelsSinkConfig.getTransactionMode()) + { + case BATCH -> + { + SinkContext sinkContext = new SinkContext("-1"); + TransContext transContext = transactionManager.getTransContext(); + sinkContext.setPixelsTransCtx(transContext); + RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder(); + TableSingleTxWriter.addUpdateData(event, builder); + List tableUpdateDataList = List.of(builder.build()); + writer.writeTrans(pixelsSinkConfig.getCaptureDatabase(), tableUpdateDataList, transContext.getTimestamp()); + transactionManager.commitTransAsync(transContext); } - }); - } - - private void handleNonTxEvent(RowChangeEvent event) { - // nonTxQueue.offer(event); - dispatchImmediately(event, null); - // event.endLatencyTimer(); + case RECORD -> + { + dispatchImmediately(event, null); + } + } } - public void shutdown() { + public void shutdown() + { dispatchExecutor.shutdown(); timeoutScheduler.shutdown(); } - public void setTxTimeoutMs(long txTimeoutMs) { + public void setTxTimeoutMs(long txTimeoutMs) + { TX_TIMEOUT_MS = txTimeoutMs; } - private static class OrderedEvent { + private static class OrderedEvent + { final RowChangeEvent event; final String table; final long collectionOrder; final long totalOrder; - OrderedEvent(RowChangeEvent event, long collectionOrder, long totalOrder) { + OrderedEvent(RowChangeEvent event, long collectionOrder, long totalOrder) + { this.event = event; this.table = event.getFullTableName(); this.collectionOrder = collectionOrder; this.totalOrder = totalOrder; } - String getTable() { + String getTable() + { return table; } - long getCollectionOrder() { + long getCollectionOrder() + { return collectionOrder; } } @Deprecated - private static class BufferedEvent { // useless + private static class BufferedEvent + { final RowChangeEvent event; final long collectionOrder; final long totalOrder; - BufferedEvent(RowChangeEvent event, long collectionOrder, long totalOrder) { + BufferedEvent(RowChangeEvent event, long collectionOrder, long totalOrder) + { this.event = event; this.collectionOrder = collectionOrder; this.totalOrder = totalOrder; } - long getTotalOrder() { + long getTotalOrder() + { return totalOrder; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java index d491fce..daff13a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java @@ -17,17 +17,21 @@ package io.pixelsdb.pixels.sink.concurrent; -public class TransactionCoordinatorFactory { +public class TransactionCoordinatorFactory +{ private static TransactionCoordinator instance; - public static synchronized TransactionCoordinator getCoordinator() { - if (instance == null) { + public static synchronized TransactionCoordinator getCoordinator() + { + if (instance == null) + { instance = new TransactionCoordinator(); } return instance; } - public static synchronized void reset() { + public static synchronized void reset() + { instance = null; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java index 9a7b989..36c0004 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java @@ -20,55 +20,102 @@ import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * This class if for * * @author AntiO2 */ -public class TransactionManager { +public class TransactionManager +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionManager.class); private final static TransactionManager instance = new TransactionManager(); private final TransService transService; private final Queue transContextQueue; private final Object batchLock = new Object(); + private final ExecutorService commitExecutor; - TransactionManager() { + TransactionManager() + { this.transService = TransService.Instance(); this.transContextQueue = new ConcurrentLinkedDeque<>(); + + this.commitExecutor = Executors.newFixedThreadPool( + 4096, + r -> + { + Thread t = new Thread(r); + t.setName("commit-trans-thread"); + t.setDaemon(true); + return t; + } + ); } - public static TransactionManager Instance() { + public static TransactionManager Instance() + { return instance; } - private void requestTransactions() { - try { - List newContexts = transService.beginTransBatch(100, false); + private void requestTransactions() + { + try + { + List newContexts = transService.beginTransBatch(1000, false); transContextQueue.addAll(newContexts); - } catch (TransException e) { + } catch (TransException e) + { throw new RuntimeException("Batch request failed", e); } } - public TransContext getTransContext() { + public TransContext getTransContext() + { TransContext ctx = transContextQueue.poll(); - if (ctx != null) { + if (ctx != null) + { return ctx; } - synchronized (batchLock) { + synchronized (batchLock) + { ctx = transContextQueue.poll(); - if (ctx == null) { + if (ctx == null) + { requestTransactions(); ctx = transContextQueue.poll(); - if (ctx == null) { + if (ctx == null) + { throw new IllegalStateException("No contexts available"); } } return ctx; } } + + public void commitTransAsync(TransContext transContext) + { + commitExecutor.submit(() -> + { + try + { + transService.commitTrans( + transContext.getTransId(), + transContext.getTimestamp() + ); + LOGGER.info("Success Commit TXID: {} TS: {}", transContext.getTransId(), transContext.getTimestamp()); + } catch (TransException e) + { + LOGGER.error("Async commit failed: transId={}", transContext.getTransId()); + e.printStackTrace(); + } + }); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java new file mode 100644 index 0000000..ff97bb8 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java @@ -0,0 +1,37 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.concurrent; + +public enum TransactionMode +{ + TRANS, + BATCH, + RECORD; + + public static TransactionMode fromValue(String value) + { + for (TransactionMode mode : values()) + { + if (mode.name().equalsIgnoreCase(value)) + { + return mode; + } + } + throw new RuntimeException(String.format("Can't convert %s to transaction mode", value)); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java index 0897b3a..4bb3e9e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java @@ -27,7 +27,9 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -public class TransactionState { + +public class TransactionState +{ private final String txId; private final long beginTs; private final Map receivedCounts = new ConcurrentHashMap<>(); @@ -35,40 +37,49 @@ public class TransactionState { private Map expectedCounts; // update this when receive END message private volatile boolean endReceived = false; - public TransactionState(String txId) { + public TransactionState(String txId) + { this.txId = txId; this.beginTs = System.currentTimeMillis(); this.expectedCounts = new HashMap<>(); } - public void addRowEvent(RowChangeEvent event) { + public void addRowEvent(RowChangeEvent event) + { rowEvents.add(event); String table = event.getTable(); - receivedCounts.compute(table, (k, v) -> { - if (v == null) { + receivedCounts.compute(table, (k, v) -> + { + if (v == null) + { return new AtomicInteger(1); - } else { + } else + { v.incrementAndGet(); return v; } }); } - public boolean isComplete() { + public boolean isComplete() + { return endReceived && expectedCounts.entrySet().stream() .allMatch(e -> receivedCounts.getOrDefault(e.getKey(), new AtomicInteger(0)).get() >= e.getValue()); } - public void markEndReceived() { + public void markEndReceived() + { this.endReceived = true; } - public boolean isExpired(long timeoutMs) { + public boolean isExpired(long timeoutMs) + { return System.currentTimeMillis() - beginTs > timeoutMs; } - public void setExpectedCounts(List dataCollectionList) { + public void setExpectedCounts(List dataCollectionList) + { this.expectedCounts = dataCollectionList.stream() .collect(Collectors.toMap( SinkProto.DataCollection::getDataCollection, @@ -76,11 +87,13 @@ public void setExpectedCounts(List dataCollectionList) )); } - public void setExpectedCounts(Map dataCollectionMap) { + public void setExpectedCounts(Map dataCollectionMap) + { this.expectedCounts = dataCollectionMap; } - public List getRowEvents() { + public List getRowEvents() + { return rowEvents; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java index 86c177b..ce8c6e8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java @@ -18,17 +18,18 @@ package io.pixelsdb.pixels.sink.config; -import static net.sourceforge.argparse4j.impl.Arguments.storeTrue; import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; import net.sourceforge.argparse4j.inf.Namespace; -public class CommandLineConfig { +public class CommandLineConfig +{ private String configPath; - public CommandLineConfig(String[] args) { + public CommandLineConfig(String[] args) + { ArgumentParser parser = ArgumentParsers .newFor("Pixels-Sink") .build() @@ -40,17 +41,20 @@ public CommandLineConfig(String[] args) { .required(false) .help("config path"); - try { + try + { Namespace res = parser.parseArgs(args); this.configPath = res.getString("config"); - } catch (ArgumentParserException e) { + } catch (ArgumentParserException e) + { parser.handleError(e); System.exit(1); } } - public String getConfigPath() { + public String getConfigPath() + { return configPath; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java new file mode 100644 index 0000000..5beced9 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java @@ -0,0 +1,39 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.config; + + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * @package: io.pixelsdb.pixels.sink + * @className: ConfigKey + * @author: AntiO2 + * @date: 2025/9/26 13:04 + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.FIELD) +public @interface ConfigKey { + String value(); + String defaultValue() default ""; + Class defaultClass() default Void.class; +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java new file mode 100644 index 0000000..7ea5ec1 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java @@ -0,0 +1,78 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.config; + + +import io.pixelsdb.pixels.sink.concurrent.TransactionMode; +import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; +import io.pixelsdb.pixels.sink.sink.RetinaWriter; + +import java.lang.reflect.Field; +import java.util.Properties; + +public class ConfigLoader { + public static void load(Properties props, Object target) { + try { + Class clazz = target.getClass(); + for (Field field : clazz.getDeclaredFields()) { + ConfigKey annotation = field.getAnnotation(ConfigKey.class); + if (annotation != null) { + String key = annotation.value(); + String value = props.getProperty(key); + if (value == null || value.isEmpty()) { + if (!annotation.defaultValue().isEmpty()) { + value = annotation.defaultValue(); + } else if (annotation.defaultClass() != Void.class) { + value = annotation.defaultClass().getName(); + } + } + Object parsed = convert(value, field.getType()); + field.setAccessible(true); + try { + field.set(target, parsed); + } catch (IllegalAccessException e) { + throw new RuntimeException("Failed to inject config for " + key, e); + } + } + } + } catch (Exception e) { + throw new RuntimeException("Failed to load config", e); + } + } + + private static Object convert(String value, Class type) { + if (type.equals(int.class) || type.equals(Integer.class)) { + return Integer.parseInt(value); + } else if (type.equals(long.class) || type.equals(Long.class)) { + return Long.parseLong(value); + } else if (type.equals(short.class) || type.equals(Short.class)) { + return Short.parseShort(value); + } else if (type.equals(boolean.class) || type.equals(Boolean.class)) { + return Boolean.parseBoolean(value); + } else if (type.equals(PixelsSinkMode.class)) { + return PixelsSinkMode.fromValue(value); + } else if (type.equals(TransactionMode.class)) { + return TransactionMode.fromValue(value); + } else if (type.equals(RetinaWriter.RetinaWriteMode.class)) { + return RetinaWriter.RetinaWriteMode.fromValue(value); + } + return value; + } +} + diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 2b9e430..61ebeff 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -17,131 +17,132 @@ package io.pixelsdb.pixels.sink.config; import io.pixelsdb.pixels.common.utils.ConfigFactory; +import io.pixelsdb.pixels.sink.concurrent.TransactionMode; +import io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer; import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; +import io.pixelsdb.pixels.sink.sink.RetinaWriter; import lombok.Getter; +import org.apache.kafka.common.serialization.StringDeserializer; import java.io.IOException; import java.util.Objects; @Getter -public class PixelsSinkConfig { - private ConfigFactory config; +public class PixelsSinkConfig +{ + private final ConfigFactory config; + @ConfigKey(value = "transaction.timeout", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_TIME_OUT) private Long transactionTimeout; + + @ConfigKey(value = "sink.mode", defaultValue = PixelsSinkDefaultConfig.SINK_MODE) private PixelsSinkMode pixelsSinkMode; + + @ConfigKey(value = "sink.retina.mode", defaultValue = PixelsSinkDefaultConfig.SINK_RETINA_MODE) + private RetinaWriter.RetinaWriteMode retinaWriteMode; + + @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) + private TransactionMode transactionMode; + + @ConfigKey(value = "sink.remote.port", defaultValue = "9090") private short remotePort; + + @ConfigKey(value = "sink.batch.size", defaultValue = "5000") private int batchSize; + + @ConfigKey(value = "sink.timeout.ms", defaultValue = "30000") private int timeoutMs; + + @ConfigKey(value = "sink.flush.interval.ms", defaultValue = "1000") private int flushIntervalMs; + + @ConfigKey(value = "sink.flush.batch.size", defaultValue = "100") + private int flushBatchSize; + + @ConfigKey(value = "sink.max.retries", defaultValue = "3") private int maxRetries; + + @ConfigKey(value = "sink.csv.enable_header", defaultValue = "false") private boolean sinkCsvEnableHeader; + + @ConfigKey(value = "sink.monitor.enable", defaultValue = "false") private boolean monitorEnabled; + + @ConfigKey(value = "sink.monitor.port", defaultValue = "9464") private short monitorPort; + + @ConfigKey(value = "sink.rpc.enable", defaultValue = "false") private boolean rpcEnable; + + @ConfigKey(value = "sink.rpc.mock.delay", defaultValue = "0") private int mockRpcDelay; + + @ConfigKey(value = "sink.trans.batch.size", defaultValue = "100") private int transBatchSize; - public PixelsSinkConfig(String configFilePath) throws IOException { - this.config = ConfigFactory.Instance(); - this.config.loadProperties(configFilePath); - parseProps(); - } + private boolean retinaEmbedded = false; - public PixelsSinkConfig(ConfigFactory config) { - this.config = config; - parseProps(); - } + @ConfigKey("topic.prefix") + private String topicPrefix; + @ConfigKey("debezium.topic.prefix") + private String debeziumTopicPrefix; - private void parseProps() { - this.pixelsSinkMode = PixelsSinkMode.fromValue(getProperty("sink.mode", PixelsSinkDefaultConfig.SINK_MODE)); - this.transactionTimeout = Long.valueOf(getProperty("transaction.timeout", TransactionConfig.DEFAULT_TRANSACTION_TIME_OUT)); - this.remotePort = parseShort(getProperty("sink.remote.port"), PixelsSinkDefaultConfig.SINK_REMOTE_PORT); - this.batchSize = parseInt(getProperty("sink.batch.size"), PixelsSinkDefaultConfig.SINK_BATCH_SIZE); - this.timeoutMs = parseInt(getProperty("sink.timeout.ms"), PixelsSinkDefaultConfig.SINK_TIMEOUT_MS); - this.flushIntervalMs = parseInt(getProperty("sink.flush.interval.ms"), PixelsSinkDefaultConfig.SINK_FLUSH_INTERVAL_MS); - this.maxRetries = parseInt(getProperty("sink.max.retries"), PixelsSinkDefaultConfig.SINK_MAX_RETRIES); - this.sinkCsvEnableHeader = parseBoolean(getProperty("sink.csv.enable_header"), PixelsSinkDefaultConfig.SINK_CSV_ENABLE_HEADER); - this.monitorEnabled = parseBoolean(getProperty("sink.monitor.enabled"), PixelsSinkDefaultConfig.SINK_MONITOR_ENABLED); - this.monitorPort = parseShort(getProperty("sink.monitor.port"), PixelsSinkDefaultConfig.SINK_MONITOR_PORT); - this.rpcEnable = parseBoolean(getProperty("sink.rpc.enable"), PixelsSinkDefaultConfig.SINK_RPC_ENABLED); - this.mockRpcDelay = parseInt(getProperty("sink.rpc.mock.delay"), PixelsSinkDefaultConfig.MOCK_RPC_DELAY); - this.transBatchSize = parseInt(getProperty("sink.trans.batch.size"), PixelsSinkDefaultConfig.TRANSACTION_BATCH_SIZE); - } + @ConfigKey("consumer.capture_database") + private String captureDatabase; - public String getTopicPrefix() { - return getProperty("topic.prefix"); - } + @ConfigKey(value = "consumer.include_tables", defaultValue = "") + private String includeTablesRaw; - public String getCaptureDatabase() { - return getProperty("consumer.capture_database"); - } + @ConfigKey("bootstrap.servers") + private String bootstrapServers; - public String[] getIncludeTables() { - String includeTables = getProperty("consumer.include_tables", ""); - return includeTables.isEmpty() ? new String[0] : includeTables.split(","); - } + @ConfigKey("group.id") + private String groupId; - public String getBootstrapServers() { - return getProperty("bootstrap.servers"); - } + @ConfigKey(value = "key.deserializer", defaultClass = StringDeserializer.class) + private String keyDeserializer; - public String getGroupId() { - return getProperty("group.id"); - } + @ConfigKey(value = "value.deserializer", defaultClass = RowChangeEventJsonDeserializer.class) + private String valueDeserializer; - public String getKeyDeserializer() { - return getProperty("key.deserializer", PixelsSinkDefaultConfig.KEY_DESERIALIZER); - } - public String getValueDeserializer() { - return getProperty("value.deserializer", PixelsSinkDefaultConfig.VALUE_DESERIALIZER); - } + @ConfigKey(value = "sink.csv.path", defaultValue = PixelsSinkDefaultConfig.CSV_SINK_PATH) + private String csvSinkPath; - public String getCsvSinkPath() { - return getProperty("sink.csv.path", PixelsSinkDefaultConfig.CSV_SINK_PATH); - } + @ConfigKey(value = "transaction.topic.suffix", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_TOPIC_SUFFIX) + private String transactionTopicSuffix; - public String getTransactionTopicSuffix() { - return getProperty("transaction.topic.suffix", TransactionConfig.DEFAULT_TRANSACTION_TOPIC_SUFFIX); - } + @ConfigKey(value = "transaction.topic.value.deserializer", + defaultClass = RowChangeEventJsonDeserializer.class) + private String transactionTopicValueDeserializer; - public String getTransactionTopicValueDeserializer() { - return getProperty("transaction.topic.value.deserializer", TransactionConfig.DEFAULT_TRANSACTION_TOPIC_VALUE_DESERIALIZER); - } + @ConfigKey(value = "transaction.topic.group_id", + defaultValue = TransactionConfig.DEFAULT_TRANSACTION_TOPIC_GROUP_ID) + private String transactionTopicGroupId; - public String getTransactionTopicGroupId() { - return getProperty("transaction.topic.group_id", TransactionConfig.DEFAULT_TRANSACTION_TOPIC_GROUP_ID); - } + @ConfigKey(value = "sink.remote.host", defaultValue = PixelsSinkDefaultConfig.SINK_REMOTE_HOST) + private String sinkRemoteHost; - public String getSinkRemoteHost() { - return getProperty("sink.remote.host", PixelsSinkDefaultConfig.SINK_REMOTE_HOST); - } - - private short parseShort(String valueStr, short defaultValue) { - return (valueStr != null) ? Short.parseShort(valueStr) : defaultValue; - } - - private int parseInt(String valueStr, int defaultValue) { - return (valueStr != null) ? Integer.parseInt(valueStr) : defaultValue; - } + @ConfigKey("sink.registry.url") + private String registryUrl; - private boolean parseBoolean(String valueStr, boolean defaultValue) { - return (valueStr != null) ? Boolean.parseBoolean(valueStr) : defaultValue; - } + @ConfigKey(value = "sink.datasource", defaultValue = PixelsSinkDefaultConfig.DATA_SOURCE) + private String dataSource; - public String getProperty(String key) { - return config.getProperty(key); + public PixelsSinkConfig(String configFilePath) throws IOException + { + this.config = ConfigFactory.Instance(); + this.config.loadProperties(configFilePath); + ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); } - public String getProperty(String key, String defaultValue) { - String value = config.getProperty(key); - if (Objects.isNull(value)) { - return defaultValue; - } - return value; + public PixelsSinkConfig(ConfigFactory config) + { + this.config = config; + ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); } - public String getRegistryUrl() { - return getProperty("sink.registry.url", ""); + public String[] getIncludeTables() { + return includeTablesRaw.isEmpty() ? new String[0] : includeTablesRaw.split(","); } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java index 9a3c354..0f3ed12 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java @@ -17,12 +17,15 @@ package io.pixelsdb.pixels.sink.config; -public final class PixelsSinkConstants { +public final class PixelsSinkConstants +{ public static final String ROW_RECORD_KAFKA_PROP_FACTORY = "row-record"; public static final String TRANSACTION_KAFKA_PROP_FACTORY = "transaction"; public static final int MONITOR_NUM = 2; public static final String SNAPSHOT_TX_PREFIX = "SNAPSHOT-"; - private PixelsSinkConstants() {} + private PixelsSinkConstants() + { + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java index ae64db3..9b649c1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java @@ -20,13 +20,12 @@ import io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -public class PixelsSinkDefaultConfig { +public class PixelsSinkDefaultConfig +{ + public static final String DATA_SOURCE = "kafka"; public static final String PROPERTIES_PATH = "pixels-sink.properties"; public static final String CSV_SINK_PATH = "./data"; - public static final String KEY_DESERIALIZER = StringDeserializer.class.getName(); // org.apache.kafka.common.serialization.StringDeserializer - public static final String VALUE_DESERIALIZER = RowChangeEventJsonDeserializer.class.getName(); - public static final String SINK_MODE = "csv"; public static final int SINK_CSV_RECORD_FLUSH = 1000; @@ -52,6 +51,7 @@ public class PixelsSinkDefaultConfig { public static final int SINK_FLUSH_INTERVAL_MS = 5000; public static final int SINK_MAX_RETRIES = 3; public static final boolean SINK_CSV_ENABLE_HEADER = false; + public static final String SINK_RETINA_MODE = "stub"; // Monitor Config public static final boolean SINK_MONITOR_ENABLED = true; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java index b762631..101e97c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java @@ -19,12 +19,14 @@ import io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer; -public class TransactionConfig { +public class TransactionConfig +{ public static final String DEFAULT_TRANSACTION_TOPIC_SUFFIX = "transaction"; public static final String DEFAULT_TRANSACTION_TOPIC_VALUE_DESERIALIZER = TransactionJsonMessageDeserializer.class.getName(); - public static final String DEFAULT_TRANSACTION_TOPIC_GROUP_ID= "transaction_consumer"; + public static final String DEFAULT_TRANSACTION_TOPIC_GROUP_ID = "transaction_consumer"; public static final String DEFAULT_TRANSACTION_TIME_OUT = "300"; public static final String DEFAULT_TRANSACTION_BATCH_SIZE = "100"; + public static final String DEFAULT_TRANSACTION_MODE = "batch"; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java index 04492a4..f3fabfb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java @@ -21,6 +21,7 @@ import java.util.Properties; -public interface KafkaPropFactory { +public interface KafkaPropFactory +{ Properties createKafkaProperties(PixelsSinkConfig config); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java index d243050..54f5d69 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java @@ -21,16 +21,20 @@ import java.util.HashMap; -public class KafkaPropFactorySelector { +public class KafkaPropFactorySelector +{ private final HashMap factories = new HashMap<>(); - public KafkaPropFactorySelector() { + public KafkaPropFactorySelector() + { factories.put(PixelsSinkConstants.TRANSACTION_KAFKA_PROP_FACTORY, new TransactionKafkaPropFactory()); factories.put(PixelsSinkConstants.ROW_RECORD_KAFKA_PROP_FACTORY, new RowRecordKafkaPropFactory()); } - public KafkaPropFactory getFactory(String type) { - if (!factories.containsKey(type)) { + public KafkaPropFactory getFactory(String type) + { + if (!factories.containsKey(type)) + { throw new IllegalArgumentException("Unknown factory type: " + type); } return factories.get(type); diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java index 77a320b..03c69de 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java @@ -22,35 +22,44 @@ import java.io.IOException; -public class PixelsSinkConfigFactory { +public class PixelsSinkConfigFactory +{ private static volatile PixelsSinkConfig instance; private static String configFilePath; private static ConfigFactory config; - private PixelsSinkConfigFactory() { + + private PixelsSinkConfigFactory() + { } - public static synchronized void initialize(String configFilePath) throws IOException { - if (instance != null) { + public static synchronized void initialize(String configFilePath) throws IOException + { + if (instance != null) + { throw new IllegalStateException("PixelsSinkConfig is already initialized!"); } instance = new PixelsSinkConfig(configFilePath); PixelsSinkConfigFactory.configFilePath = configFilePath; } - public static synchronized void initialize(ConfigFactory config) { + public static synchronized void initialize(ConfigFactory config) + { PixelsSinkConfigFactory.config = config; instance = new PixelsSinkConfig(config); } - public static PixelsSinkConfig getInstance() { - if (instance == null) { + public static PixelsSinkConfig getInstance() + { + if (instance == null) + { throw new IllegalStateException("PixelsSinkConfig is not initialized! Call initialize() first."); } return instance; } - public static synchronized void reset() { + public static synchronized void reset() + { instance = null; configFilePath = null; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java index d1d4336..cb052e2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java @@ -22,20 +22,29 @@ import java.util.Properties; -public class RowRecordKafkaPropFactory implements KafkaPropFactory{ +public class RowRecordKafkaPropFactory implements KafkaPropFactory +{ + static Properties getCommonKafkaProperties(PixelsSinkConfig config) + { + Properties kafkaProperties = new Properties(); + kafkaProperties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.getBootstrapServers()); + kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, config.getKeyDeserializer()); + kafkaProperties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + return kafkaProperties; + } + @Override - public Properties createKafkaProperties(PixelsSinkConfig config) { + public Properties createKafkaProperties(PixelsSinkConfig config) + { Properties kafkaProperties = getCommonKafkaProperties(config); kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, config.getValueDeserializer()); kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, config.getGroupId()); - return kafkaProperties; - } - static Properties getCommonKafkaProperties(PixelsSinkConfig config) { - Properties kafkaProperties = new Properties(); - kafkaProperties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.getBootstrapServers()); - kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, config.getKeyDeserializer()); - kafkaProperties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + kafkaProperties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "30000"); + kafkaProperties.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000"); + kafkaProperties.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "600000"); + kafkaProperties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "500"); + return kafkaProperties; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java index 4221c2c..1c678a3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java @@ -24,10 +24,12 @@ import static io.pixelsdb.pixels.sink.config.factory.RowRecordKafkaPropFactory.getCommonKafkaProperties; -public class TransactionKafkaPropFactory implements KafkaPropFactory{ +public class TransactionKafkaPropFactory implements KafkaPropFactory +{ @Override - public Properties createKafkaProperties(PixelsSinkConfig config) { - Properties kafkaProperties = getCommonKafkaProperties(config); + public Properties createKafkaProperties(PixelsSinkConfig config) + { + Properties kafkaProperties = getCommonKafkaProperties(config); kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, config.getTransactionTopicValueDeserializer()); kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, config.getTransactionTopicGroupId() + "-" + config.getGroupId()); return kafkaProperties; diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java index 2c39ec8..5614ffb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java @@ -20,13 +20,19 @@ import com.google.protobuf.ByteString; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; import org.apache.avro.AvroRuntimeException; import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.source.SourceRecord; import java.util.Arrays; -public class DeserializerUtil { - static RowChangeEvent buildErrorEvent(String topic, byte[] rawData, Exception error) { +public class DeserializerUtil +{ + static RowChangeEvent buildErrorEvent(String topic, byte[] rawData, Exception error) throws SinkException + { SinkProto.ErrorInfo errorInfo = SinkProto.ErrorInfo.newBuilder() .setMessage(error.getMessage()) .setStackTrace(Arrays.toString(error.getStackTrace())) @@ -38,57 +44,78 @@ static RowChangeEvent buildErrorEvent(String topic, byte[] rawData, Exception er .setTsMs(System.currentTimeMillis()) .build(); - return new RowChangeEvent(record) { + return new RowChangeEvent(record, null) + { @Override - public boolean hasError() { + public boolean hasError() + { return true; } @Override - public SinkProto.ErrorInfo getErrorInfo() { + public SinkProto.ErrorInfo getErrorInfo() + { return errorInfo; } @Override - public String getTopic() { + public String getTopic() + { return topic; } }; } - static public SinkProto.TransactionStatus getStatusSafely(GenericRecord record, String field) { + static public SinkProto.TransactionStatus getStatusSafely(T record, String field) + { String statusString = getStringSafely(record, field); - if (statusString.equals("BEGIN")) { + if (statusString.equals("BEGIN")) + { return SinkProto.TransactionStatus.BEGIN; } - if (statusString.equals("END")) { + if (statusString.equals("END")) + { return SinkProto.TransactionStatus.END; } return SinkProto.TransactionStatus.UNRECOGNIZED; } - static public String getStringSafely(GenericRecord record, String field) { + public static Object getFieldSafely(T record, String field) { try { - Object value = record.get(field); - return value != null ? value.toString() : ""; - } catch (AvroRuntimeException e) { - return ""; + if (record instanceof GenericRecord avro) { + return avro.get(field); + } else if (record instanceof Struct struct) { + return struct.get(field); + } else if (record instanceof SourceRecord sourceRecord) { + return ((Struct) sourceRecord.value()).get(field); + } + } catch (Exception e) { + return null; } + return null; } - static public Long getLongSafely(GenericRecord record, String field) { - try { - Object value = record.get(field); - return value instanceof Number ? ((Number) value).longValue() : 0L; - } catch (AvroRuntimeException e) { - return 0L; - } + public static String getStringSafely(T record, String field) { + Object value = getFieldSafely(record, field); + return value != null ? value.toString() : ""; + } + + public static Long getLongSafely(T record, String field) { + Object value = getFieldSafely(record, field); + return value instanceof Number ? ((Number) value).longValue() : 0L; + } + + public static Integer getIntSafely(T record, String field) { + Object value = getFieldSafely(record, field); + return value instanceof Number ? ((Number) value).intValue() : 0; } - static public SinkProto.OperationType getOperationType(String op) { + static public SinkProto.OperationType getOperationType(String op) + { op = op.toLowerCase(); - return switch (op) { + return switch (op) + { case "c" -> SinkProto.OperationType.INSERT; case "u" -> SinkProto.OperationType.UPDATE; case "d" -> SinkProto.OperationType.DELETE; @@ -96,4 +123,22 @@ static public SinkProto.OperationType getOperationType(String op) { default -> throw new IllegalArgumentException(String.format("Can't convert %s to operation type", op)); }; } + + static public boolean hasBeforeValue(SinkProto.OperationType op) + { + return op == SinkProto.OperationType.DELETE || op == SinkProto.OperationType.UPDATE; + } + + static public boolean hasAfterValue(SinkProto.OperationType op) + { + return op != SinkProto.OperationType.DELETE; + } + + static public String getTransIdPrefix(String originTransID) + { + return originTransID.contains(":") + ? originTransID.substring(0, originTransID.indexOf(":")) + : originTransID; + } + } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java index dc25482..1ee7662 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java @@ -24,8 +24,9 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.common.serialization.Deserializer; @@ -33,14 +34,16 @@ import java.util.HashMap; import java.util.Map; -public class RowChangeEventAvroDeserializer implements Deserializer { +public class RowChangeEventAvroDeserializer implements Deserializer +{ private final AvroKafkaDeserializer avroDeserializer = new AvroKafkaDeserializer<>(); private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); private final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); @Override - public void configure(Map configs, boolean isKey) { + public void configure(Map configs, boolean isKey) + { Map enrichedConfig = new HashMap<>(configs); enrichedConfig.put(SerdeConfig.REGISTRY_URL, config.getRegistryUrl()); enrichedConfig.put(SerdeConfig.CHECK_PERIOD_MS, SerdeConfig.CHECK_PERIOD_MS_DEFAULT); @@ -48,14 +51,17 @@ public void configure(Map configs, boolean isKey) { } @Override - public RowChangeEvent deserialize(String topic, byte[] data) { - try { + public RowChangeEvent deserialize(String topic, byte[] data) + { + try + { MetricsFacade.getInstance().addRawData(data.length); GenericRecord avroRecord = avroDeserializer.deserialize(topic, data); Schema avroSchema = avroRecord.getSchema(); RowChangeEvent rowChangeEvent = convertToRowChangeEvent(avroRecord, avroSchema); return rowChangeEvent; - } catch (Exception e) { + } catch (Exception e) + { e.printStackTrace(); return null; // throw new SerializationException("Avro deserialization failed", e); @@ -63,30 +69,41 @@ public RowChangeEvent deserialize(String topic, byte[] data) { } } - private void registerSchema(String topic, Schema avroSchema) { + private void registerSchema(String topic, Schema avroSchema) + { } - private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema schema) { + private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema schema) throws SinkException + { SinkProto.OperationType op = parseOperationType(avroRecord); SinkProto.RowRecord.Builder recordBuilder = SinkProto.RowRecord.newBuilder() .setOp(op) .setTsMs(DeserializerUtil.getLongSafely(avroRecord, "ts_ms")); - if (avroRecord.get("source") != null) { + if (avroRecord.get("source") != null) + { //TODO: 这里看下怎么处理,如果没有source信息,其实可以通过topic推出schema和table信息。 parseSourceInfo((GenericRecord) avroRecord.get("source"), recordBuilder.getSourceBuilder()); } String sourceSchema = recordBuilder.getSource().getDb(); String sourceTable = recordBuilder.getSource().getTable(); - TypeDescription typeDescription = tableMetadataRegistry.parseTypeDescription(avroRecord, sourceSchema, sourceTable); + TypeDescription typeDescription = null; + try + { + typeDescription = tableMetadataRegistry.getTypeDescription(sourceSchema, sourceTable); + } catch (SinkException e) + { + throw new RuntimeException(e); + } // TableMetadata tableMetadata = tableMetadataRegistry.loadTableMetadata(sourceSchema, sourceTable); recordBuilder.setBefore(parseRowData(avroRecord.get("before"), typeDescription)); recordBuilder.setAfter(parseRowData(avroRecord.get("after"), typeDescription)); - if (avroRecord.get("transaction") != null) { + if (avroRecord.get("transaction") != null) + { parseTransactionInfo((GenericRecord) avroRecord.get("transaction"), recordBuilder.getTransactionBuilder()); } @@ -94,25 +111,31 @@ private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema return new RowChangeEvent(recordBuilder.build(), typeDescription); } - private SinkProto.OperationType parseOperationType(GenericRecord record) { + private SinkProto.OperationType parseOperationType(GenericRecord record) + { String op = DeserializerUtil.getStringSafely(record, "op"); - try { + try + { return DeserializerUtil.getOperationType(op); - } catch (IllegalArgumentException e) { + } catch (IllegalArgumentException e) + { return SinkProto.OperationType.UNRECOGNIZED; } } - private SinkProto.RowValue.Builder parseRowData(Object data, TypeDescription typeDescription) { + private SinkProto.RowValue.Builder parseRowData(Object data, TypeDescription typeDescription) + { SinkProto.RowValue.Builder builder = SinkProto.RowValue.newBuilder(); - if (data instanceof GenericRecord rowData) { + if (data instanceof GenericRecord rowData) + { RowDataParser rowDataParser = new RowDataParser(typeDescription); // TODO make it static? rowDataParser.parse(rowData, builder); } return builder; } - private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder builder) { + private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder builder) + { builder.setVersion(DeserializerUtil.getStringSafely(source, "version")) .setConnector(DeserializerUtil.getStringSafely(source, "connector")) .setName(DeserializerUtil.getStringSafely(source, "name")) @@ -128,8 +151,9 @@ private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder } private void parseTransactionInfo(GenericRecord transaction, - SinkProto.TransactionInfo.Builder builder) { - builder.setId(DeserializerUtil.getStringSafely(transaction, "id")) + SinkProto.TransactionInfo.Builder builder) + { + builder.setId(DeserializerUtil.getTransIdPrefix(DeserializerUtil.getStringSafely(transaction, "id"))) .setTotalOrder(DeserializerUtil.getLongSafely(transaction, "total_order")) .setDataCollectionOrder(DeserializerUtil.getLongSafely(transaction, "data_collection_order")); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java index bf5f002..44802ba 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java @@ -22,62 +22,64 @@ import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; -public class RowChangeEventJsonDeserializer implements Deserializer { +public class RowChangeEventJsonDeserializer implements Deserializer +{ private static final Logger logger = LoggerFactory.getLogger(RowChangeEventJsonDeserializer.class); private static final ObjectMapper objectMapper = new ObjectMapper(); + private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); @Override - public RowChangeEvent deserialize(String topic, byte[] data) { - if (data == null || data.length == 0) { + public RowChangeEvent deserialize(String topic, byte[] data) + { + if (data == null || data.length == 0) + { logger.debug("Received empty message from topic: {}", topic); return null; } MetricsFacade.getInstance().addRawData(data.length); - try { + try + { JsonNode rootNode = objectMapper.readTree(data); - JsonNode schemaNode = rootNode.path("schema"); JsonNode payloadNode = rootNode.path("payload"); SinkProto.OperationType opType = parseOperationType(payloadNode); - TypeDescription schema = getSchema(schemaNode, opType); - return buildRowRecord(payloadNode, schema, opType); - } catch (Exception e) { + return buildRowRecord(payloadNode, opType); + } catch (Exception e) + { logger.error("Failed to deserialize message from topic {}: {}", topic, e.getMessage()); - return DeserializerUtil.buildErrorEvent(topic, data, e); + return null; } } - private SinkProto.OperationType parseOperationType(JsonNode payloadNode) { + private SinkProto.OperationType parseOperationType(JsonNode payloadNode) + { String opCode = payloadNode.path("op").asText(""); return DeserializerUtil.getOperationType(opCode); } - // TODO: cache schema - private TypeDescription getSchema(JsonNode schemaNode, SinkProto.OperationType opType) { - switch (opType) { - case DELETE: - return SchemaDeserializer.parseFromBeforeOrAfter(schemaNode, "before"); - case INSERT: - case UPDATE: - case SNAPSHOT: - return SchemaDeserializer.parseFromBeforeOrAfter(schemaNode, "after"); - case UNRECOGNIZED: - throw new IllegalArgumentException("Operation type is unknown. Check op"); - } - return null; + @Deprecated + private TypeDescription getSchema(JsonNode schemaNode, SinkProto.OperationType opType) + { + return switch (opType) + { + case DELETE -> SchemaDeserializer.parseFromBeforeOrAfter(schemaNode, "before"); + case INSERT, UPDATE, SNAPSHOT -> SchemaDeserializer.parseFromBeforeOrAfter(schemaNode, "after"); + case UNRECOGNIZED -> throw new IllegalArgumentException("Operation type is unknown. Check op"); + }; } private RowChangeEvent buildRowRecord(JsonNode payloadNode, - TypeDescription schema, - SinkProto.OperationType opType) { + SinkProto.OperationType opType) throws SinkException + { SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); @@ -86,43 +88,55 @@ private RowChangeEvent buildRowRecord(JsonNode payloadNode, .setTsUs(payloadNode.path("ts_us").asLong()) .setTsNs(payloadNode.path("ts_ns").asLong()); - Map beforeData = parseDataFields(payloadNode, schema, opType, "before"); - Map afterData = parseDataFields(payloadNode, schema, opType, "after"); - if (payloadNode.has("source")) { - builder.setSource(parseSourceInfo(payloadNode.get("source"))); + String schemaName; + String tableName; + if (payloadNode.has("source")) + { + SinkProto.SourceInfo.Builder sourceInfoBuilder = parseSourceInfo(payloadNode.get("source")); + schemaName = sourceInfoBuilder.getDb(); // Notice we use the schema + tableName = sourceInfoBuilder.getTable(); + builder.setSource(sourceInfoBuilder); + } else + { + throw new IllegalArgumentException("Missing source field in row record"); } - if (payloadNode.hasNonNull("transaction")) { + TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); + RowDataParser rowDataParser = new RowDataParser(typeDescription); + if (payloadNode.hasNonNull("transaction")) + { builder.setTransaction(parseTransactionInfo(payloadNode.get("transaction"))); } - // RowChangeEvent event = new RowChangeEvent(builder.build(), schema, opType, beforeData, afterData); - RowChangeEvent event = new RowChangeEvent(builder.build()); - event.initIndexKey(); - return event; - } + if (DeserializerUtil.hasBeforeValue(opType)) + { + SinkProto.RowValue.Builder beforeBuilder = builder.getBeforeBuilder(); + rowDataParser.parse(payloadNode.get("before"), beforeBuilder); + builder.setBefore(beforeBuilder); + } - private Map parseDataFields(JsonNode payloadNode, - TypeDescription schema, - SinkProto.OperationType opType, - String dataField) { - RowDataParser parser = new RowDataParser(schema); + if (DeserializerUtil.hasAfterValue(opType)) + { - JsonNode dataNode = payloadNode.get(dataField); - if (dataNode != null && !dataNode.isNull()) { - return parser.parse(dataNode, opType); + SinkProto.RowValue.Builder afterBuilder = builder.getAfterBuilder(); + rowDataParser.parse(payloadNode.get("after"), afterBuilder); + builder.setAfter(afterBuilder); } - return null; - } - private JsonNode resolveDataNode(JsonNode payloadNode, SinkProto.OperationType opType) { - return opType == SinkProto.OperationType.DELETE ? - payloadNode.get("before") : - payloadNode.get("after"); - } + RowChangeEvent event = new RowChangeEvent(builder.build(), typeDescription); + try + { + event.initIndexKey(); + } catch (SinkException e) + { + logger.warn("Row change event {}: Init index key failed", event); + } + return event; + } - private SinkProto.SourceInfo parseSourceInfo(JsonNode sourceNode) { + private SinkProto.SourceInfo.Builder parseSourceInfo(JsonNode sourceNode) + { return SinkProto.SourceInfo.newBuilder() .setVersion(sourceNode.path("version").asText()) .setConnector(sourceNode.path("connector").asText()) @@ -137,25 +151,17 @@ private SinkProto.SourceInfo parseSourceInfo(JsonNode sourceNode) { .setTable(sourceNode.path("table").asText()) .setTxId(sourceNode.path("txId").asLong()) .setLsn(sourceNode.path("lsn").asLong()) - .setXmin(sourceNode.path("xmin").asLong()) - .build(); + .setXmin(sourceNode.path("xmin").asLong()); } - private SinkProto.TransactionInfo parseTransactionInfo(JsonNode txNode) { + private SinkProto.TransactionInfo parseTransactionInfo(JsonNode txNode) + { return SinkProto.TransactionInfo.newBuilder() - .setId(txNode.path("id").asText()) + .setId(DeserializerUtil.getTransIdPrefix(txNode.path("id").asText())) .setTotalOrder(txNode.path("total_order").asLong()) .setDataCollectionOrder(txNode.path("data_collection_order").asLong()) .build(); } - - private boolean hasAfterData(SinkProto.OperationType op) { - return op != SinkProto.OperationType.DELETE; - } - - private boolean hasBeforeData(SinkProto.OperationType op) { - return op == SinkProto.OperationType.DELETE || op == SinkProto.OperationType.UPDATE; - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java new file mode 100644 index 0000000..0aeb2af --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java @@ -0,0 +1,132 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.deserializer; + + +import com.fasterxml.jackson.databind.JsonNode; +import io.pixelsdb.pixels.core.TypeDescription; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.logging.Logger; + +/** + * @package: io.pixelsdb.pixels.sink.deserializer + * @className: RowChangeEventStructDeserializer + * @author: AntiO2 + * @date: 2025/9/26 12:00 + */ +public class RowChangeEventStructDeserializer +{ + private static final Logger LOGGER = Logger.getLogger(RowChangeEventStructDeserializer.class.getName()); + private static final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + + public static RowChangeEvent convertToRowChangeEvent(SourceRecord sourceRecord) throws SinkException + { + Struct value = (Struct) sourceRecord.value(); + String op = value.getString("op"); + SinkProto.OperationType operationType = DeserializerUtil.getOperationType(op); + return buildRowRecord(value, operationType); + } + + private static RowChangeEvent buildRowRecord(Struct value, + SinkProto.OperationType opType) throws SinkException + { + + SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); + + builder.setOp(opType); + + String schemaName; + String tableName; + try { + Struct source = value.getStruct("source"); + SinkProto.SourceInfo.Builder sourceInfoBuilder = parseSourceInfo(source); + schemaName = sourceInfoBuilder.getDb(); // Notice we use the schema + tableName = sourceInfoBuilder.getTable(); + builder.setSource(sourceInfoBuilder); + } catch (DataException e) + { + LOGGER.warning("Missing source field in row record"); + throw new SinkException(e); + } + + TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); + RowDataParser rowDataParser = new RowDataParser(typeDescription); + + try { + Struct transaction = value.getStruct("transaction"); + SinkProto.TransactionInfo transactionInfo = parseTransactionInfo(transaction); + builder.setTransaction(transactionInfo); + } catch (DataException e) + { + LOGGER.warning("Missing transaction field in row record"); + } + + if (DeserializerUtil.hasBeforeValue(opType)) + { + SinkProto.RowValue.Builder beforeBuilder = builder.getBeforeBuilder(); + rowDataParser.parse(value.getStruct("before"), beforeBuilder); + builder.setBefore(beforeBuilder); + } + + if (DeserializerUtil.hasAfterValue(opType)) + { + + SinkProto.RowValue.Builder afterBuilder = builder.getAfterBuilder(); + rowDataParser.parse(value.getStruct("after"), afterBuilder); + builder.setAfter(afterBuilder); + } + + RowChangeEvent event = new RowChangeEvent(builder.build(), typeDescription); + return event; + } + + private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) { + return SinkProto.SourceInfo.newBuilder() + .setVersion(DeserializerUtil.getStringSafely(source, "version")) + .setConnector(DeserializerUtil.getStringSafely(source, "connector")) + .setName(DeserializerUtil.getStringSafely(source, "name")) + .setTsMs(DeserializerUtil.getLongSafely(source, "ts_ms")) + .setSnapshot(DeserializerUtil.getStringSafely(source, "snapshot")) + .setDb(DeserializerUtil.getStringSafely(source, "db")) + .setSequence(DeserializerUtil.getStringSafely(source, "sequence")) + .setTsUs(DeserializerUtil.getLongSafely(source, "ts_us")) + .setTsNs(DeserializerUtil.getLongSafely(source, "ts_ns")) + .setSchema(DeserializerUtil.getStringSafely(source, "schema")) + .setTable(DeserializerUtil.getStringSafely(source, "table")) + .setTxId(DeserializerUtil.getLongSafely(source, "txId")) + .setLsn(DeserializerUtil.getLongSafely(source, "lsn")) + .setXmin(DeserializerUtil.getLongSafely(source, "xmin")); + } + + private static SinkProto.TransactionInfo parseTransactionInfo(T txNode) { + return SinkProto.TransactionInfo.newBuilder() + .setId(DeserializerUtil.getTransIdPrefix( + DeserializerUtil.getStringSafely(txNode, "id"))) + .setTotalOrder(DeserializerUtil.getLongSafely(txNode, "total_order")) + .setDataCollectionOrder(DeserializerUtil.getLongSafely(txNode, "data_collection_order")) + .build(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java index 43f9b26..de79028 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java @@ -18,142 +18,360 @@ package io.pixelsdb.pixels.sink.deserializer; import com.fasterxml.jackson.databind.JsonNode; +import com.google.protobuf.ByteString; import io.pixelsdb.pixels.core.PixelsProto; import io.pixelsdb.pixels.core.TypeDescription; -import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.util.DateUtil; import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.time.LocalDate; import java.util.Base64; import java.util.Collections; -import java.util.HashMap; +import java.util.Date; import java.util.Map; -class RowDataParser { + +class RowDataParser +{ private final TypeDescription schema; - public RowDataParser(TypeDescription schema) { + public RowDataParser(TypeDescription schema) + { this.schema = schema; } - public Map parse(JsonNode dataNode, SinkProto.OperationType operation) { - if (dataNode.isNull() && operation == SinkProto.OperationType.DELETE) { - return parseDeleteRecord(); - } - return parseNode(dataNode, schema); - } - public void parse(GenericRecord record, SinkProto.RowValue.Builder builder) { - for (int i = 0; i < schema.getFieldNames().size(); i++) { + public void parse(GenericRecord record, SinkProto.RowValue.Builder builder) + { + for (int i = 0; i < schema.getFieldNames().size(); i++) + { String fieldName = schema.getFieldNames().get(i); TypeDescription fieldType = schema.getChildren().get(i); builder.addValues(parseValue(record, fieldName, fieldType).build()); - // result.put(fieldName, parseValue(node.get(fieldName), fieldType)); } } - private Map parseNode(JsonNode node, TypeDescription schema) { - Map result = new HashMap<>(); - for (int i = 0; i < schema.getFieldNames().size(); i++) { + public void parse(JsonNode node, SinkProto.RowValue.Builder builder) + { + for (int i = 0; i < schema.getFieldNames().size(); i++) + { String fieldName = schema.getFieldNames().get(i); TypeDescription fieldType = schema.getChildren().get(i); - result.put(fieldName, parseValue(node.get(fieldName), fieldType)); + builder.addValues(parseValue(node.get(fieldName), fieldName, fieldType).build()); + } + } + + + public void parse(Struct record, SinkProto.RowValue.Builder builder) + { + for (int i = 0; i < schema.getFieldNames().size(); i++) + { + String fieldName = schema.getFieldNames().get(i); + Field field = record.schema().field(fieldName); + Schema.Type fieldType = field.schema().type(); + builder.addValues(parseValue(record.get(fieldName), fieldName, fieldType).build()); } - return result; } + private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fieldName, TypeDescription type) + { + if (valueNode == null || valueNode.isNull()) + { + return SinkProto.ColumnValue.newBuilder() + .setName(fieldName) + .setValue(ByteString.EMPTY); + } - private Object parseValue(JsonNode valueNode, TypeDescription type) { - if (valueNode.isNull()) return null; + SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); + columnValueBuilder.setName(fieldName); - switch (type.getCategory()) { + switch (type.getCategory()) + { case INT: - return valueNode.asInt(); + { + int value = valueNode.asInt(); + byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(value).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); + break; + } case LONG: - return valueNode.asLong(); + { + long value = valueNode.asLong(); + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); + break; + } + case CHAR: + { + String text = valueNode.asText(); + byte[] bytes = new byte[] { (byte) text.charAt(0) }; + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder() + .setKind(PixelsProto.Type.Kind.STRING)); + break; + } + case VARCHAR: case STRING: - return valueNode.asText().trim(); + case VARBINARY: + { + String value = valueNode.asText().trim(); + columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); + break; + } case DECIMAL: - return parseDecimal(valueNode, type); - case DATE: - return parseDate(valueNode); - case STRUCT: - return parseNode(valueNode, type); + { + String value = parseDecimal(valueNode, type).toString(); + columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder() + .setKind(PixelsProto.Type.Kind.DECIMAL) + .setDimension(type.getPrecision()) + .setScale(type.getScale())); + break; + } case BINARY: - return parseBinary(valueNode); + { + String base64 = valueNode.asText(); // assume already base64 encoded + columnValueBuilder.setValue(ByteString.copyFrom(base64, StandardCharsets.UTF_8)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); + break; + } + case STRUCT: + { + // You can recursively parse fields in a struct here + throw new UnsupportedOperationException("STRUCT parsing not yet implemented"); + } + case DOUBLE: + { + double value = valueNode.asDouble(); + long longBits = Double.doubleToLongBits(value); + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(longBits).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); + break; + } + case FLOAT: + { + float value = (float) valueNode.asDouble(); + int intBits = Float.floatToIntBits(value); + byte[] bytes = ByteBuffer.allocate(4).putInt(intBits).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); + break; + } + case DATE: + { + int isoDate = valueNode.asInt(); + byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(isoDate).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder() + .setKind(PixelsProto.Type.Kind.DATE)); + break; + } + case TIMESTAMP: + { + long timestamp = valueNode.asLong(); + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(timestamp).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder() + .setKind(PixelsProto.Type.Kind.DATE)); + break; + } default: - throw new IllegalArgumentException("Unsupported type: " + type); + throw new IllegalArgumentException("Unsupported type: " + type.getCategory()); } + + return columnValueBuilder; } - private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String filedName, TypeDescription filedType) { + @Deprecated // TODO: use bit + private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fieldName, TypeDescription fieldType) + { SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); - columnValueBuilder.setName(filedName); - switch (filedType.getCategory()) { - case INT: { - int value = (int) record.get(filedName); - columnValueBuilder.setValue(RetinaProto.ColumnValue.newBuilder().setNumberVal(Integer.toString(value))); + columnValueBuilder.setName(fieldName); + + Object raw = record.get(fieldName); + if (raw == null) + { + columnValueBuilder.setValue(ByteString.EMPTY); + return columnValueBuilder; + } + + switch (fieldType.getCategory()) + { + case INT: + { + int value = (int) raw; + columnValueBuilder.setValue(ByteString.copyFrom(Integer.toString(value), StandardCharsets.UTF_8)); columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } - case LONG: { - long value = (long) record.get(filedName); - columnValueBuilder.setValue(RetinaProto.ColumnValue.newBuilder().setNumberVal(Long.toString(value))); + case LONG: + { + long value = (long) raw; + columnValueBuilder.setValue(ByteString.copyFrom(Long.toString(value), StandardCharsets.UTF_8)); columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } - case STRING: { - String value = (String) record.get(filedName).toString(); - columnValueBuilder.setValue(RetinaProto.ColumnValue.newBuilder().setStringVal(value)); + case STRING: + { + String value = raw.toString(); + columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } - case DECIMAL: { + + case DECIMAL: + { + ByteBuffer buffer = (ByteBuffer) raw; + String decimalStr = new String(buffer.array(), StandardCharsets.UTF_8).trim(); + columnValueBuilder.setValue(ByteString.copyFrom(decimalStr, StandardCharsets.UTF_8)); columnValueBuilder.setType(PixelsProto.Type.newBuilder() .setKind(PixelsProto.Type.Kind.DECIMAL) - .setDimension(filedType.getDimension()) - .setScale(filedType.getScale()) - .build()); - columnValueBuilder.setValue(RetinaProto.ColumnValue.newBuilder().setNumberVal( - new String(((ByteBuffer) record.get(filedName)).array()))); - break; - } -// case DATE: -// return parseDate(valueNode); -// case STRUCT: -// return parseNode(valueNode, type); -// case BINARY: -// return parseBinary(valueNode); + .setDimension(fieldType.getPrecision()) + .setScale(fieldType.getScale())); + break; + } + + case DATE: + { + int epochDay = (int) raw; + String isoDate = LocalDate.ofEpochDay(epochDay).toString(); // e.g., "2025-07-03" + columnValueBuilder.setValue(ByteString.copyFrom(isoDate, StandardCharsets.UTF_8)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DATE)); + break; + } + + case BINARY: + { + ByteBuffer buffer = (ByteBuffer) raw; + // encode as hex or base64 if needed, otherwise just dump as UTF-8 string if it's meant to be readable + String base64 = Base64.getEncoder().encodeToString(buffer.array()); + columnValueBuilder.setValue(ByteString.copyFrom(base64, StandardCharsets.UTF_8)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); + break; + } default: - throw new IllegalArgumentException("Unsupported type: " + filedType.getCategory()); + throw new IllegalArgumentException("Unsupported type: " + fieldType.getCategory()); + } + + return columnValueBuilder; + } + + private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName, Schema.Type type) + { + // TODO(AntiO2) support pixels type + if (record == null) + { + return SinkProto.ColumnValue.newBuilder() + .setName(fieldName) + .setValue(ByteString.EMPTY); } + + SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); + columnValueBuilder.setName(fieldName); + + switch (type) + { + case INT8: + case INT16: + case INT32: + { + int value = (Integer) record; + byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(value).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); + break; + } + case INT64: + { + long value = (Long) record; + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); + break; + } + case BYTES: + { + byte[] bytes = (byte[]) record; + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BYTE)); + break; + } + case BOOLEAN: + case STRING: + { + String value = (String) record; + columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); + break; + } + case STRUCT: + { + // You can recursively parse fields in a struct here + throw new UnsupportedOperationException("STRUCT parsing not yet implemented"); + } + case FLOAT64: + { + double value = (double) record; + long longBits = Double.doubleToLongBits(value); + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(longBits).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); + break; + } + case FLOAT32: + { + float value = (float) record; + int intBits = Float.floatToIntBits(value); + byte[] bytes = ByteBuffer.allocate(4).putInt(intBits).array(); + columnValueBuilder.setValue(ByteString.copyFrom(bytes)); + columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); + break; + } + default: + throw new IllegalArgumentException("Unsupported type: " + type); + } + return columnValueBuilder; } - private Map parseDeleteRecord() { + private Map parseDeleteRecord() + { return Collections.singletonMap("__deleted", true); } - BigDecimal parseDecimal(JsonNode node, TypeDescription type) { + BigDecimal parseDecimal(JsonNode node, TypeDescription type) + { byte[] bytes = Base64.getDecoder().decode(node.asText()); int scale = type.getScale(); return new BigDecimal(new BigInteger(bytes), scale); } - private LocalDate parseDate(JsonNode node) { + private LocalDate parseDate(JsonNode node) + { return LocalDate.ofEpochDay(node.asLong()); } - private byte[] parseBinary(JsonNode node) { - try { + private byte[] parseBinary(JsonNode node) + { + try + { return node.binaryValue(); - } catch (IOException e) { + } catch (IOException e) + { throw new RuntimeException("Binary parsing failed", e); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowRecordDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowRecordDeserializer.java deleted file mode 100644 index 1a9c266..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowRecordDeserializer.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.deserializer; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.util.JsonFormat; -import io.pixelsdb.pixels.sink.SinkProto; -import org.apache.kafka.common.serialization.Deserializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Map; - -public class RowRecordDeserializer implements Deserializer { - - private static final Logger LOGGER = LoggerFactory.getLogger(RowRecordDeserializer.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final JsonFormat.Parser PROTO_PARSER = JsonFormat.parser().ignoringUnknownFields(); - - static SinkProto.RowRecord parseRowRecord(Map rawMessage) throws IOException { - SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); - String json = OBJECT_MAPPER.writeValueAsString(rawMessage.get("payload")); - //TODO optimize - return getRowRecord(json, builder); - } - - private static SinkProto.RowRecord getRowRecord(String json, SinkProto.RowRecord.Builder builder) throws InvalidProtocolBufferException { - PROTO_PARSER.merge(json, builder); - return builder.build(); - } - - @Override - public SinkProto.RowRecord deserialize(String topic, byte[] data) { - if (data == null || data.length == 0) { - return null; - } - try { - Map rawMessage = OBJECT_MAPPER.readValue(data, Map.class); - return parseRowRecord(rawMessage); - } catch (IOException e) { - LOGGER.error("Failed to deserialize row record message", e); - throw new RuntimeException("Deserialization error", e); - } - } -} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java index 97e7802..df66bb3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java @@ -25,29 +25,37 @@ import java.util.Iterator; import java.util.Map; -public class SchemaDeserializer { - public static TypeDescription parseFromBeforeOrAfter(JsonNode schemaNode, String fieldName) { +public class SchemaDeserializer +{ + public static TypeDescription parseFromBeforeOrAfter(JsonNode schemaNode, String fieldName) + { JsonNode beforeAfterSchema = findSchemaField(schemaNode, fieldName); - if (beforeAfterSchema == null) { + if (beforeAfterSchema == null) + { throw new IllegalArgumentException("Field '" + fieldName + "' not found in schema"); } return parseStruct(beforeAfterSchema.get("fields")); } - private static JsonNode findSchemaField(JsonNode schemaNode, String targetField) { + private static JsonNode findSchemaField(JsonNode schemaNode, String targetField) + { Iterator fields = schemaNode.get("fields").elements(); - while (fields.hasNext()) { + while (fields.hasNext()) + { JsonNode field = fields.next(); - if (targetField.equals(field.get("field").asText())) { + if (targetField.equals(field.get("field").asText())) + { return field; } } return null; } - static TypeDescription parseStruct(JsonNode fields) { + static TypeDescription parseStruct(JsonNode fields) + { TypeDescription structType = TypeDescription.createStruct(); - fields.forEach(field -> { + fields.forEach(field -> + { String name = field.get("field").asText(); TypeDescription fieldType = parseFieldType(field); structType.addField(name, fieldType); @@ -55,15 +63,19 @@ static TypeDescription parseStruct(JsonNode fields) { return structType; } - static TypeDescription parseFieldType(JsonNode fieldNode) { - if (!fieldNode.has("type")) { + static TypeDescription parseFieldType(JsonNode fieldNode) + { + if (!fieldNode.has("type")) + { throw new IllegalArgumentException("Field is missing required 'type' property"); } String typeName = fieldNode.get("type").asText(); String logicalType = fieldNode.has("name") ? fieldNode.get("name").asText() : null; - if (logicalType != null) { - switch (logicalType) { + if (logicalType != null) + { + switch (logicalType) + { case "org.apache.kafka.connect.data.Decimal": int precision = Integer.parseInt(fieldNode.get("parameters").get("connect.decimal.precision").asText()); int scale = Integer.parseInt(fieldNode.get("parameters").get("scale").asText()); @@ -73,7 +85,8 @@ static TypeDescription parseFieldType(JsonNode fieldNode) { } } - switch (typeName) { + switch (typeName) + { case "int64": return TypeDescription.createLong(); case "int32": @@ -87,9 +100,11 @@ static TypeDescription parseFieldType(JsonNode fieldNode) { } } - public static TypeDescription parseFromBeforeOrAfter(Schema schemaNode, String fieldName) { + public static TypeDescription parseFromBeforeOrAfter(Schema schemaNode, String fieldName) + { Schema.Field filed = schemaNode.getField(fieldName); - if (filed == null) { + if (filed == null) + { throw new IllegalArgumentException("Can't find field in avro schema: " + fieldName); } @@ -98,18 +113,22 @@ public static TypeDescription parseFromBeforeOrAfter(Schema schemaNode, String f } - public static TypeDescription parseFromAvroSchema(Schema avroSchema) { + public static TypeDescription parseFromAvroSchema(Schema avroSchema) + { return parseAvroType(avroSchema, new HashMap<>()); } - private static TypeDescription parseAvroType(Schema schema, Map cache) { + private static TypeDescription parseAvroType(Schema schema, Map cache) + { String schemaKey = schema.getFullName() + ":" + schema.hashCode(); - if (cache.containsKey(schemaKey)) { + if (cache.containsKey(schemaKey)) + { return cache.get(schemaKey); } TypeDescription typeDesc; - switch (schema.getType()) { + switch (schema.getType()) + { case RECORD: typeDesc = parseAvroRecord(schema, cache); break; @@ -130,38 +149,48 @@ private static TypeDescription parseAvroType(Schema schema, Map cache) { + private static TypeDescription parseAvroRecord(Schema schema, Map cache) + { TypeDescription structType = TypeDescription.createStruct(); - for (Schema.Field field : schema.getFields()) { + for (Schema.Field field : schema.getFields()) + { TypeDescription fieldType = parseAvroType(field.schema(), cache); structType.addField(field.name(), fieldType); } return structType; } - private static TypeDescription parseAvroUnion(Schema schema, Map cache) { - for (Schema type : schema.getTypes()) { - if (type.getType() != Schema.Type.NULL) { + private static TypeDescription parseAvroUnion(Schema schema, Map cache) + { + for (Schema type : schema.getTypes()) + { + if (type.getType() != Schema.Type.NULL) + { return parseAvroType(type, cache); } } throw new IllegalArgumentException("Invalid union type: " + schema); } - private static TypeDescription parseAvroArray(Schema schema, Map cache) { + private static TypeDescription parseAvroArray(Schema schema, Map cache) + { throw new RuntimeException("Doesn't support Array"); } - private static TypeDescription parseAvroMap(Schema schema, Map cache) { + private static TypeDescription parseAvroMap(Schema schema, Map cache) + { throw new RuntimeException("Doesn't support Map"); } - private static TypeDescription parseAvroPrimitive(Schema schema) { + private static TypeDescription parseAvroPrimitive(Schema schema) + { String logicalType = schema.getLogicalType() != null ? schema.getLogicalType().getName() : null; - if (logicalType != null) { - switch (logicalType) { + if (logicalType != null) + { + switch (logicalType) + { case "decimal": return TypeDescription.createDecimal( (Integer) (schema.getObjectProp("precision")), @@ -176,7 +205,8 @@ private static TypeDescription parseAvroPrimitive(Schema schema) { } } - switch (schema.getType()) { + switch (schema.getType()) + { case LONG: return TypeDescription.createLong(); case INT: diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java index db75449..72d6b24 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java @@ -22,7 +22,7 @@ import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; @@ -32,13 +32,15 @@ import java.util.HashMap; import java.util.Map; -public class TransactionAvroMessageDeserializer implements Deserializer { +public class TransactionAvroMessageDeserializer implements Deserializer +{ private static final Logger logger = LoggerFactory.getLogger(TransactionAvroMessageDeserializer.class); private final AvroKafkaDeserializer avroDeserializer = new AvroKafkaDeserializer<>(); private final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); @Override - public void configure(Map configs, boolean isKey) { + public void configure(Map configs, boolean isKey) + { Map enrichedConfig = new HashMap<>(configs); enrichedConfig.put(SerdeConfig.REGISTRY_URL, config.getRegistryUrl()); enrichedConfig.put(SerdeConfig.CHECK_PERIOD_MS, SerdeConfig.CHECK_PERIOD_MS_DEFAULT); @@ -46,46 +48,27 @@ public void configure(Map configs, boolean isKey) { } @Override - public SinkProto.TransactionMetadata deserialize(String topic, byte[] bytes) { - if (bytes == null || bytes.length == 0) { + public SinkProto.TransactionMetadata deserialize(String topic, byte[] bytes) + { + if (bytes == null || bytes.length == 0) + { return null; } - try { + try + { MetricsFacade.getInstance().addRawData(bytes.length); GenericRecord avroRecord = avroDeserializer.deserialize(topic, bytes); - return convertToTransactionMetadata(avroRecord); - } catch (Exception e) { + return TransactionStructMessageDeserializer.convertToTransactionMetadata(avroRecord); + } catch (Exception e) + { logger.error("Avro deserialization failed for topic {}: {}", topic, e.getMessage()); throw new SerializationException("Failed to deserialize Avro message", e); } } - private SinkProto.TransactionMetadata convertToTransactionMetadata(GenericRecord record) { - SinkProto.TransactionMetadata.Builder builder = - SinkProto.TransactionMetadata.newBuilder(); - builder.setStatus(DeserializerUtil.getStatusSafely(record, "status")) - .setId(DeserializerUtil.getStringSafely(record, "id")) - .setEventCount(DeserializerUtil.getLongSafely(record, "event_count")) - .setTimestamp(DeserializerUtil.getLongSafely(record, "ts_ms")); - - if (record.get("data_collections") != null) { - Iterable collections = (Iterable) record.get("data_collections"); - for (Object item : collections) { - if (item instanceof GenericRecord collectionRecord) { - SinkProto.DataCollection.Builder collectionBuilder = - SinkProto.DataCollection.newBuilder(); - collectionBuilder.setDataCollection(DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); - collectionBuilder.setEventCount(DeserializerUtil.getLongSafely(collectionRecord, "event_count")); - builder.addDataCollections(collectionBuilder); - } - } - } - - return builder.build(); - } - @Override - public void close() { + public void close() + { Deserializer.super.close(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java index 8923155..8ce20d9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java @@ -20,7 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.protobuf.util.JsonFormat; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,32 +28,40 @@ import java.io.IOException; import java.util.Map; -public class TransactionJsonMessageDeserializer implements Deserializer { +public class TransactionJsonMessageDeserializer implements Deserializer +{ private static final Logger LOGGER = LoggerFactory.getLogger(TransactionJsonMessageDeserializer.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final JsonFormat.Parser PROTO_PARSER = JsonFormat.parser().ignoringUnknownFields(); @Override - public SinkProto.TransactionMetadata deserialize(String topic, byte[] data) { - if (data == null || data.length == 0) { + public SinkProto.TransactionMetadata deserialize(String topic, byte[] data) + { + if (data == null || data.length == 0) + { return null; } MetricsFacade.getInstance().addRawData(data.length); - try { + try + { Map rawMessage = OBJECT_MAPPER.readValue(data, Map.class); return parseTransactionMetadata(rawMessage); - } catch (IOException e) { + } catch (IOException e) + { LOGGER.error("Failed to deserialize transaction message", e); throw new RuntimeException("Deserialization error", e); } } - private SinkProto.TransactionMetadata parseTransactionMetadata(Map rawMessage) throws IOException { + private SinkProto.TransactionMetadata parseTransactionMetadata(Map rawMessage) throws IOException + { SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); String json = OBJECT_MAPPER.writeValueAsString(rawMessage.get("payload")); PROTO_PARSER.merge(json, builder); + builder.setId(DeserializerUtil.getTransIdPrefix(builder.getId())); + return builder.build(); } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java new file mode 100644 index 0000000..358a8f3 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java @@ -0,0 +1,78 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.deserializer; + + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.protobuf.util.JsonFormat; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; + +/** + * @package: io.pixelsdb.pixels.sink.deserializer + * @className: TransactionStructMessageDeserializer + * @author: AntiO2 + * @date: 2025/9/26 12:42 + */ +public class TransactionStructMessageDeserializer +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionStructMessageDeserializer.class); + + @SuppressWarnings("unchecked") + public static SinkProto.TransactionMetadata convertToTransactionMetadata(T record) { + SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); + + builder.setStatus(DeserializerUtil.getStatusSafely(record, "status")) + .setId(DeserializerUtil.getTransIdPrefix( + DeserializerUtil.getStringSafely(record, "id"))) + .setEventCount(DeserializerUtil.getLongSafely(record, "event_count")) + .setTimestamp(DeserializerUtil.getLongSafely(record, "ts_ms")); + + Object collections = DeserializerUtil.getFieldSafely(record, "data_collections"); + if (collections instanceof Iterable) { + for (Object item : (Iterable) collections) { + if (item instanceof GenericRecord collectionRecord) { + SinkProto.DataCollection.Builder collectionBuilder = SinkProto.DataCollection.newBuilder(); + collectionBuilder.setDataCollection( + DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); + collectionBuilder.setEventCount( + DeserializerUtil.getLongSafely(collectionRecord, "event_count")); + builder.addDataCollections(collectionBuilder); + } else if (item instanceof Struct collectionRecord) { + SinkProto.DataCollection.Builder collectionBuilder = SinkProto.DataCollection.newBuilder(); + collectionBuilder.setDataCollection( + DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); + collectionBuilder.setEventCount( + DeserializerUtil.getLongSafely(collectionRecord, "event_count")); + builder.addDataCollections(collectionBuilder); + } + } + } + + return builder.build(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 8a592c7..f4d22ac 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -18,218 +18,270 @@ package io.pixelsdb.pixels.sink.event; import com.google.protobuf.ByteString; -import io.pixelsdb.pixels.common.metadata.domain.SecondaryIndex; +import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.index.IndexProto; -import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadata; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import io.prometheus.client.Summary; import lombok.Getter; import lombok.Setter; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; -public class RowChangeEvent { +public class RowChangeEvent +{ @Getter private final SinkProto.RowRecord rowRecord; - private IndexProto.IndexKey indexKey; - private boolean isIndexKeyInited; - @Setter - private SecondaryIndex indexInfo; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + @Getter + private final TypeDescription schema; /** * timestamp from pixels transaction server */ @Setter @Getter private long timeStamp; - - @Getter - private final TypeDescription schema; - @Getter private String topic; - @Getter private TableMetadata tableMetadata = null; - - private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private Summary.Timer latencyTimer; private Map beforeValueMap; private Map afterValueMap; + @Getter + private IndexProto.IndexKey beforeKey; + @Getter + private IndexProto.IndexKey afterKey; - public RowChangeEvent(SinkProto.RowRecord rowRecord) { + public RowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException + { this.rowRecord = rowRecord; - this.schema = null; + this.schema = TableMetadataRegistry.Instance().getTypeDescription(getSchemaName(), getTable()); + initColumnValueMap(); + initIndexKey(); } - - public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) { + public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) throws SinkException + { this.rowRecord = rowRecord; this.schema = schema; + initColumnValueMap(); + // initIndexKey(); } - private void initColumnValueMap() { - if (hasBeforeData()) { + private void initColumnValueMap() + { + if (hasBeforeData()) + { + this.beforeValueMap = new HashMap<>(); initColumnValueMap(rowRecord.getBefore(), beforeValueMap); } - if (hasAfterData()) { + if (hasAfterData()) + { + this.afterValueMap = new HashMap<>(); initColumnValueMap(rowRecord.getAfter(), afterValueMap); } } - private void initColumnValueMap(SinkProto.RowValue rowValue, Map map) { + private void initColumnValueMap(SinkProto.RowValue rowValue, Map map) + { rowValue.getValuesList().forEach( - column -> { + column -> + { map.put(column.getName(), column); } ); } - public void setTimeStamp(long timeStamp) { - this.timeStamp = timeStamp; - } - - public void setIndexInfo(SecondaryIndex indexInfo) { - this.indexInfo = indexInfo; - } + public void initIndexKey() throws SinkException + { + this.tableMetadata = TableMetadataRegistry.Instance().getMetadata( + this.rowRecord.getSource().getDb(), + this.rowRecord.getSource().getTable()); - public IndexProto.IndexKey getIndexKey() { - if (!isIndexKeyInited) { - initIndexKey(); + if (!this.tableMetadata.hasPrimaryIndex()) + { + return; + } + if (hasBeforeData()) + { + this.beforeKey = generateIndexKey(tableMetadata, beforeValueMap); } - return indexKey; - } - public void initIndexKey() { - if (!hasAfterData()) { - // We do not need to generate an index key for insert request - return; + if (hasAfterData()) + { + this.afterKey = generateIndexKey(tableMetadata, afterValueMap); } + } - this.tableMetadata = TableMetadataRegistry.Instance().getMetadata( - this.rowRecord.getSource().getDb(), - this.rowRecord.getSource().getTable()); + private IndexProto.IndexKey generateIndexKey(TableMetadata tableMetadata, Map rowValue) + { List keyColumnNames = tableMetadata.getKeyColumnNames(); - ByteBuffer byteBuffer = ByteBuffer.allocate(1024); + SinglePointIndex index = tableMetadata.getIndex(); + int len = keyColumnNames.size(); + List keyColumnValues = new ArrayList<>(len); + int keySize = 0; + for (String keyColumnName : keyColumnNames) + { + ByteString value = rowValue.get(keyColumnName).getValue(); + keyColumnValues.add(value); + keySize += value.size(); + } + keySize += Long.BYTES + (len + 1) * 2; // table id + index key + ByteBuffer byteBuffer = ByteBuffer.allocate(keySize); - for (int i = 0; i < keyColumnNames.size(); i++) { - String name = keyColumnNames.get(i); - byteBuffer.put(afterValueMap.get(name).getValue().toByteArray()); - if (i < keyColumnNames.size() - 1) { - byteBuffer.putChar(':'); - } + byteBuffer.putLong(index.getTableId()).putChar(':'); + for (ByteString value : keyColumnValues) + { + byteBuffer.put(value.toByteArray()); + byteBuffer.putChar(':'); } - this.indexKey = IndexProto.IndexKey.newBuilder() + return IndexProto.IndexKey.newBuilder() .setTimestamp(timeStamp) - .setKey(ByteString.copyFrom(byteBuffer)) - .setIndexId(indexInfo.getId()) + .setKey(ByteString.copyFrom(byteBuffer.rewind())) + .setIndexId(index.getId()) + .setTableId(tableMetadata.getTable().getId()) .build(); - isIndexKeyInited = true; } - - // TODO change - public RetinaProto.ColumnValue getBeforePk() { - return rowRecord.getBefore().getValues(0).getValue(); - } - - public RetinaProto.ColumnValue getAfterPk() { - return rowRecord.getBefore().getValues(0).getValue(); - } - - public String getSourceTable() { + public String getSourceTable() + { return rowRecord.getSource().getTable(); } - public SinkProto.TransactionInfo getTransaction() { + public SinkProto.TransactionInfo getTransaction() + { return rowRecord.getTransaction(); } - public String getTable() { + public String getTable() + { return rowRecord.getSource().getTable(); } - public String getFullTableName() { - return getSchemaName() + "." + getTable(); + public String getFullTableName() + { + // TODO(AntiO2): In postgresql, data collection uses schemaName as prefix, while MySQL uses DB as prefix. + return rowRecord.getSource().getSchema() + "." + rowRecord.getSource().getTable(); + // return getSchemaName() + "." + getTable(); } + // TODO(AntiO2): How to Map Schema Names Between Source DB and Pixels - public String getSchemaName() { + public String getSchemaName() + { return rowRecord.getSource().getDb(); // return rowRecord.getSource().getSchema(); } - public boolean hasError() { + public boolean hasError() + { return false; } - public SinkProto.ErrorInfo getErrorInfo() { + public SinkProto.ErrorInfo getErrorInfo() + { return rowRecord.getError(); } - public String getDb() { + public String getDb() + { return rowRecord.getSource().getDb(); } - public boolean isDelete() { + public boolean isDelete() + { return getOp() == SinkProto.OperationType.DELETE; } - public boolean isInsert() { + public boolean isInsert() + { return getOp() == SinkProto.OperationType.INSERT; } - public boolean isSnapshot() { + public boolean isSnapshot() + { return getOp() == SinkProto.OperationType.SNAPSHOT; } - public boolean isUpdate() { + + public boolean isUpdate() + { return getOp() == SinkProto.OperationType.UPDATE; } - public boolean hasBeforeData() { + public boolean hasBeforeData() + { return isUpdate() || isDelete(); } - public boolean hasAfterData() { + public boolean hasAfterData() + { return isUpdate() || isInsert() || isSnapshot(); } - public Long getTimeStampUs() { + public Long getTimeStampUs() + { return rowRecord.getTsUs(); } - public int getPkId() { - return tableMetadata.getPkId(); - } - - public void startLatencyTimer() { + public void startLatencyTimer() + { this.latencyTimer = metricsFacade.startProcessLatencyTimer(); } - public void endLatencyTimer() { - if (latencyTimer != null) { + public void endLatencyTimer() + { + if (latencyTimer != null) + { this.latencyTimer.close(); } } - public SinkProto.OperationType getOp() { + public SinkProto.OperationType getOp() + { return rowRecord.getOp(); } - public SinkProto.RowValue getBeforeData() { + public SinkProto.RowValue getBefore() + { return rowRecord.getBefore(); } - public SinkProto.RowValue getAfterData() { + public SinkProto.RowValue getAfter() + { return rowRecord.getAfter(); } + + public List getAfterData() + { + List colValues = rowRecord.getAfter().getValuesList(); + List colValueList = new ArrayList<>(colValues.size()); + for (SinkProto.ColumnValue col : colValues) + { + colValueList.add(col.getValue()); + } + return colValueList; + } + + @Override + public String toString() + { + String sb = "RowChangeEvent{" + + rowRecord.getSource().getDb() + + "." + rowRecord.getSource().getTable() + + rowRecord.getTransaction().getId(); + return sb; + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java new file mode 100644 index 0000000..549c8bd --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java @@ -0,0 +1,57 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.processor.TableProcessor; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEnginePipelineManager + * @author: AntiO2 + * @date: 2025/9/26 10:44 + */ +public class TableEnginePipelineManager extends TablePipelineManager +{ + private final Map pipelines = new ConcurrentHashMap<>(); + + public void routeRecord(SchemaTableName schemaTableName, SourceRecord record) { + TableEventEngineProvider tableEventEngineProvider = pipelines.computeIfAbsent(schemaTableName, + k-> + { + TableEventEngineProvider newProvider = createPipeline(k); + TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> + new TableProcessor(newProvider, schemaTableName)); + tableProcessor.run(); + return newProvider; + }); + tableEventEngineProvider.put(record); + } + + private TableEventEngineProvider createPipeline(SchemaTableName schemaTableName) { + TableEventEngineProvider pipeline = new TableEventEngineProvider(schemaTableName); + pipeline.start(); + return pipeline; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java new file mode 100644 index 0000000..7e273e5 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java @@ -0,0 +1,90 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.logging.Logger; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEventEngineProvider + * @author: AntiO2 + * @date: 2025/9/26 10:45 + */ +public class TableEventEngineProvider implements TableEventProvider { + private final Logger LOGGER = Logger.getLogger(TableEventEngineProvider.class.getName()); + private final SchemaTableName schemaTableName; + + private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(10000); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + private final Thread processorThread; + + public TableEventEngineProvider(SchemaTableName schemaTableName) + { + this.schemaTableName = schemaTableName; + this.processorThread = new Thread(this::processLoop, "TableEventEngineProvider-" + schemaTableName.getTableName()); + } + + @Override + public BlockingQueue getSourceEventQueue() + { + return eventQueue; + } + + public void start() { + processorThread.start(); + } + + public void put(SourceRecord record) { + try { + rawEventQueue.put(record); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + private void processLoop() { + while (true) { + try { + SourceRecord record = rawEventQueue.take(); + RowChangeEvent rowChangeEvent = null; + try + { + rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(record); + } catch (SinkException e) + { + LOGGER.warning(e.getMessage()); + continue; + } + eventQueue.put(rowChangeEvent); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java new file mode 100644 index 0000000..c57604b --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java @@ -0,0 +1,32 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.event; + + +import java.util.concurrent.BlockingQueue; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEventProvider + * @author: AntiO2 + * @date: 2025/9/26 07:47 + */ +public interface TableEventProvider +{ + BlockingQueue getSourceEventQueue(); +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java new file mode 100644 index 0000000..90ff559 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java @@ -0,0 +1,40 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.processor.TableMonitor; +import io.pixelsdb.pixels.sink.processor.TableProcessor; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TablePipelineManager + * @author: AntiO2 + * @date: 2025/9/26 10:44 + */ +abstract public class TablePipelineManager +{ + protected final ExecutorService executorService = Executors.newCachedThreadPool(); + protected final Map activeTableProcessors = new ConcurrentHashMap<>(); +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java new file mode 100644 index 0000000..fc13b92 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java @@ -0,0 +1,58 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.deserializer.TransactionStructMessageDeserializer; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TransactionEventEngineProvider + * @author: AntiO2 + * @date: 2025/9/25 13:20 + */ +public class TransactionEventEngineProvider implements TransactionEventProvider { + + public static final TransactionEventEngineProvider INSTANCE = new TransactionEventEngineProvider(); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + + private TransactionEventEngineProvider() {} + + public static TransactionEventEngineProvider getInstance() + { + return INSTANCE; + } + + @Override + public BlockingQueue getEventQueue() { + return eventQueue; + } + + public SinkProto.TransactionMetadata convert(SourceRecord sourceRecord) + { + Struct value = (Struct) sourceRecord.value(); + return TransactionStructMessageDeserializer.convertToTransactionMetadata(value); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java new file mode 100644 index 0000000..aa014af --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java @@ -0,0 +1,107 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.errors.WakeupException; + +import java.io.Closeable; +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TransactionEventKafkaProvider + * @author: AntiO2 + * @date: 2025/9/25 13:40 + */ +public class TransactionEventKafkaProvider implements TransactionEventProvider, Runnable, Closeable +{ + private final AtomicBoolean running = new AtomicBoolean(true); + private final String transactionTopic; + private final KafkaConsumer consumer; + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + private TransactionEventKafkaProvider() + { + Properties kafkaProperties = new Properties(); + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + + this.transactionTopic = pixelsSinkConfig.getTopicPrefix() + "." + pixelsSinkConfig.getTransactionTopicSuffix(); + this.consumer = new KafkaConsumer<>(kafkaProperties); + } + + @Override + public BlockingQueue getEventQueue() + { + return eventQueue; + } + + @Override + public void run() + { + consumer.subscribe(Collections.singletonList(transactionTopic)); + while (running.get()) + { + try + { + + ConsumerRecords records = + consumer.poll(Duration.ofMillis(1000)); + + for (ConsumerRecord record : records) + { + try + { + eventQueue.put(record.value()); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + } + } catch (WakeupException e) + { + if (running.get()) + { + // LOGGER.warn("Consumer wakeup unexpectedly", e); + } + } catch (Exception e) + { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + } + + @Override + public void close() throws IOException + { + running.set(true); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java new file mode 100644 index 0000000..5fb4ae4 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java @@ -0,0 +1,34 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.sink.SinkProto; + +import java.util.concurrent.BlockingQueue; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TransactionEventProvider + * @author: AntiO2 + * @date: 2025/9/25 13:37 + */ +public interface TransactionEventProvider +{ + BlockingQueue getEventQueue(); +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java b/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java new file mode 100644 index 0000000..1160cf5 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java @@ -0,0 +1,38 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.sink.exception; + +public class SinkException extends Exception +{ + public SinkException(String msg) + { + super(msg); + } + + public SinkException(String message, Throwable cause) + { + super(message, cause); + } + + public SinkException(Throwable cause) + { + super(cause); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/SchemaCache.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/SchemaCache.java deleted file mode 100644 index 8d7bef9..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/SchemaCache.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.metadata; - -import io.pixelsdb.pixels.core.TypeDescription; - -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.function.Supplier; - -public class SchemaCache { - private static final SchemaCache INSTANCE = new SchemaCache(); - private final ConcurrentMap cache = new ConcurrentHashMap<>(); - - private SchemaCache() { - } - - ; - - public static SchemaCache getInstance() { - return INSTANCE; - } - - public TypeDescription computeIfAbsent(TableMetadataKey tableMetadataKey, Supplier supplier) { - return cache.computeIfAbsent(tableMetadataKey, k -> supplier.get()); - } - - // public TypeDescription get(String topic) { -// return cache.get(topic); -// } -} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java index 87fe8ac..bae02a5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java @@ -17,34 +17,67 @@ package io.pixelsdb.pixels.sink.metadata; +import io.pixelsdb.pixels.common.exception.MetadataException; import io.pixelsdb.pixels.common.metadata.domain.Column; -import io.pixelsdb.pixels.common.metadata.domain.SecondaryIndex; +import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.common.metadata.domain.Table; +import io.pixelsdb.pixels.core.TypeDescription; import lombok.Getter; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; @Getter -public class TableMetadata { +public class TableMetadata +{ private final Table table; - private final SecondaryIndex index; - private final List keyColumnNames; - + private final SinglePointIndex index; + private final TypeDescription typeDescription; private final List columns; + private final List keyColumnNames; - public TableMetadata(Table table, SecondaryIndex index, List columns) { + public TableMetadata(Table table, SinglePointIndex index, List columns) throws MetadataException + { this.table = table; this.index = index; this.columns = columns; - keyColumnNames = new LinkedList<>(); - List keyColumnIds = index.getKeyColumns().getKeyColumnIds(); - for (Integer keyColumnId : keyColumnIds) { - keyColumnNames.add(columns.get(keyColumnId).getName()); + this.keyColumnNames = new LinkedList<>(); + List columnNames = columns.stream().map(Column::getName).collect(Collectors.toList()); + List columnTypes = columns.stream().map(Column::getType).collect(Collectors.toList()); + typeDescription = TypeDescription.createSchemaFromStrings(columnNames, columnTypes); + if (index != null) + { + Map columnMap = new HashMap<>(); + for (Column column : columns) + { + columnMap.put(column.getId(), column); + } + + for (Integer keyColumnId : index.getKeyColumns().getKeyColumnIds()) + { + Column column = columnMap.get(keyColumnId.longValue()); + if (column != null) + { + keyColumnNames.add(column.getName()); + } else + { + throw new MetadataException("Cant find key column id: " + keyColumnId + " in table " + + table.getName() + " schema id is " + table.getSchemaId()); + } + } } } - public int getPkId() { - return index.getKeyColumns().getKeyColumnIds().get(0); + public boolean hasPrimaryIndex() + { + return index != null; + } + + public long getPrimaryIndexKeyId() + { + return index.getId(); } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataKey.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataKey.java deleted file mode 100644 index 3cb3b7d..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataKey.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.metadata; - -import lombok.Getter; - -import java.util.Objects; - -public final class TableMetadataKey { - @Getter - private final String schemaName; - @Getter - private final String tableName; - private final int hash; - - public TableMetadataKey(String schemaName, String tableName) { - this.schemaName = schemaName.toLowerCase(); - this.tableName = tableName.toLowerCase(); - this.hash = Objects.hash(this.schemaName, this.tableName); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - TableMetadataKey that = (TableMetadataKey) o; - return schemaName.equals(that.schemaName) && - tableName.equals(that.tableName); - } - - @Override - public int hashCode() { - return hash; - } -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index 1d9b6b6..d5afd0c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -19,33 +19,38 @@ import io.pixelsdb.pixels.common.exception.MetadataException; import io.pixelsdb.pixels.common.metadata.MetadataService; +import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.common.metadata.domain.Column; -import io.pixelsdb.pixels.common.metadata.domain.SecondaryIndex; +import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.common.metadata.domain.Table; import io.pixelsdb.pixels.core.TypeDescription; -import io.pixelsdb.pixels.sink.deserializer.SchemaDeserializer; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -public class TableMetadataRegistry { +public class TableMetadataRegistry +{ + private static final Logger logger = LoggerFactory.getLogger(TableMetadataRegistry.class); private static final MetadataService metadataService = MetadataService.Instance(); private static volatile TableMetadataRegistry instance; - private final ConcurrentMap registry = new ConcurrentHashMap<>(); - private final ConcurrentMap typeDescriptionConcurrentMap = new ConcurrentHashMap<>(); - private final SchemaCache schemaCache = SchemaCache.getInstance(); + private final ConcurrentMap registry = new ConcurrentHashMap<>(); - private TableMetadataRegistry() { + private TableMetadataRegistry() + { } - public static TableMetadataRegistry Instance() { - if (instance == null) { - synchronized (TableMetadataRegistry.class) { - if (instance == null) { + public static TableMetadataRegistry Instance() + { + if (instance == null) + { + synchronized (TableMetadataRegistry.class) + { + if (instance == null) + { instance = new TableMetadataRegistry(); } } @@ -53,45 +58,58 @@ public static TableMetadataRegistry Instance() { return instance; } - public TableMetadata getMetadata(String schema, String table) { - TableMetadataKey key = new TableMetadataKey(schema, table); - return registry.computeIfAbsent(key, k -> loadTableMetadata(schema, table)); + public TableMetadata getMetadata(String schema, String table) throws SinkException + { + SchemaTableName key = new SchemaTableName(schema, table); + if (!registry.containsKey(key)) + { + logger.info("Registry doesn't contain {}", key); + TableMetadata metadata = loadTableMetadata(schema, table); + registry.put(key, metadata); + } + return registry.get(key); + } + + + public TypeDescription getTypeDescription(String schemaName, String tableName) throws SinkException + { + return getMetadata(schemaName, tableName).getTypeDescription(); + } + + public List getKeyColumnsName(String schemaName, String tableName) throws SinkException + { + return getMetadata(schemaName, tableName).getKeyColumnNames(); + } + + public long getPrimaryIndexKeyId(String schemaName, String tableName) throws SinkException + { + return getMetadata(schemaName, tableName).getPrimaryIndexKeyId(); } - public TableMetadata loadTableMetadata(String schemaName, String tableName) { - try { + private TableMetadata loadTableMetadata(String schemaName, String tableName) throws SinkException + { + try + { + logger.info("Metadata Cache miss: {} {}", schemaName, tableName); Table table = metadataService.getTable(schemaName, tableName); - SecondaryIndex index = metadataService.getSecondaryIndex(table.getId()); - /* - TODO(Lizn): we only use unique index? - */ - if (!index.isUnique()) { - throw new MetadataException("Non Unique Index is not supported"); + SinglePointIndex index = null; + try + { + index = metadataService.getPrimaryIndex(table.getId()); + } catch (MetadataException e) + { + logger.warn("Could not get primary index for table {}", tableName, e); + } + + if (!index.isUnique()) + { + throw new MetadataException("Non Unique Index is not supported, Schema:" + schemaName + " Table: " + tableName); } List tableColumns = metadataService.getColumns(schemaName, tableName, false); return new TableMetadata(table, index, tableColumns); - } catch (MetadataException e) { - throw new RuntimeException(e); + } catch (MetadataException e) + { + throw new SinkException(e); } } - - public TypeDescription getTypeDescription(String schema, String table) { - return typeDescriptionConcurrentMap.get(new TableMetadataKey(schema, table)); - } - - /** - * parse typeDescription from avro record and cache it. - * - * @param record - * @return - */ - public TypeDescription parseTypeDescription(GenericRecord record, String sourceSchema, String sourceTable) { - Schema schema = ((GenericData.Record) record).getSchema().getField("before").schema().getTypes().get(1); - TableMetadataKey tableMetadataKey = new TableMetadataKey(sourceSchema, sourceTable); - TypeDescription typeDescription = typeDescriptionConcurrentMap.computeIfAbsent( - tableMetadataKey, - key -> SchemaDeserializer.parseFromAvroSchema(schema) - ); - return typeDescription; - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/monitor/TransactionMonitor.java b/src/main/java/io/pixelsdb/pixels/sink/monitor/TransactionMonitor.java deleted file mode 100644 index bd2f3a0..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/monitor/TransactionMonitor.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.monitor; - -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.errors.WakeupException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.Duration; -import java.util.Collections; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; - -public class TransactionMonitor implements Runnable, StoppableMonitor { - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionMonitor.class); - - private final String transactionTopic; - private final KafkaConsumer consumer; - private final TransactionCoordinator transactionCoordinator; - private final AtomicBoolean running = new AtomicBoolean(true); - - public TransactionMonitor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) { - this.transactionTopic = pixelsSinkConfig.getTopicPrefix() + "." + pixelsSinkConfig.getTransactionTopicSuffix(); - this.consumer = new KafkaConsumer<>(kafkaProperties); - this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); - } - - @Override - public void run() { - try { - consumer.subscribe(Collections.singletonList(transactionTopic)); - LOGGER.info("Started transaction monitor for topic: {}", transactionTopic); - - while (running.get()) { - try { - ConsumerRecords records = - consumer.poll(Duration.ofMillis(1000)); - - for (ConsumerRecord record : records) { - SinkProto.TransactionMetadata transaction = record.value(); - LOGGER.debug("Processing transaction event: {}", transaction.getId()); - transactionCoordinator.processTransactionEvent(transaction); - } - } catch (WakeupException e) { - if (running.get()) { - LOGGER.warn("Consumer wakeup unexpectedly", e); - } - } - } - } finally { - closeResources(); - LOGGER.info("Transaction monitor stopped"); - } - } - - @Override - public void stopMonitor() { - LOGGER.info("Stopping transaction monitor"); - running.set(false); - consumer.wakeup(); - } - - private void closeResources() { - try { - if (consumer != null) { - consumer.close(Duration.ofSeconds(5)); - LOGGER.debug("Kafka consumer closed"); - } - } catch (Exception e) { - LOGGER.warn("Error closing Kafka consumer", e); - } - } -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java new file mode 100644 index 0000000..5bb3d02 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java @@ -0,0 +1,31 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.processor; + + +/** + * @package: io.pixelsdb.pixels.sink.processor + * @className: MainProcessor + * @author: AntiO2 + * @date: 2025/9/26 13:45 + */ +public interface MainProcessor extends StoppableProcessor +{ + public void start(); + public boolean isRunning(); +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/monitor/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java similarity index 75% rename from src/main/java/io/pixelsdb/pixels/sink/monitor/MetricsFacade.java rename to src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java index 8c02db3..6f142b8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/monitor/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.monitor; +package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -24,7 +24,8 @@ import io.prometheus.client.Counter; import io.prometheus.client.Summary; -public class MetricsFacade { +public class MetricsFacade +{ private static MetricsFacade instance; private final boolean enabled; private final Counter tableChangeCounter; @@ -32,15 +33,31 @@ public class MetricsFacade { private final Counter transactionCounter; private final Summary processingLatency; private final Counter rawDataThroughputCounter; + private final Counter debeziumEventCounter; + private final Counter rowEventCounter; private final Summary transServiceLatency; private final Summary indexServiceLatency; private final Summary retinaServiceLatency; private final Summary writerLatency; private final Summary totalLatency; - private MetricsFacade(boolean enabled) { + + private MetricsFacade(boolean enabled) + { this.enabled = enabled; - if (enabled) { + if (enabled) + { + this.debeziumEventCounter = Counter.build() + .name("debezium_event_total") + .help("Debezium Event Total") + .register(); + + this.rowEventCounter = Counter.build() + .name("row_event_total") + .help("Debezium Row Event Total") + .register(); + + this.tableChangeCounter = Counter.build() .name("sink_table_changes_total") .help("Total processed table changes") @@ -118,7 +135,10 @@ private MetricsFacade(boolean enabled) { .quantile(0.99, 0.001) .register(); - } else { + } else + { + this.debeziumEventCounter = null; + this.rowEventCounter = null; this.rowChangeCounter = null; this.transactionCounter = null; this.processingLatency = null; @@ -132,65 +152,98 @@ private MetricsFacade(boolean enabled) { } } - public static synchronized void initialize() { + private static synchronized void initialize() + { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - if (instance == null) { + if (instance == null) + { instance = new MetricsFacade(config.isMonitorEnabled()); } } - public static MetricsFacade getInstance() { - if (instance == null) { + public static MetricsFacade getInstance() + { + if (instance == null) + { initialize(); } return instance; } - public void recordRowChange(String table, SinkProto.OperationType operation) { + public void recordDebeziumEvent() + { + if(enabled && debeziumEventCounter != null) + { + debeziumEventCounter.inc(); + } + } + + public void recordRowChange(String table, SinkProto.OperationType operation) + { recordRowChange(table, operation, 1); } - public void recordRowChange(String table, SinkProto.OperationType operation, int rows) { - if (enabled && rowChangeCounter != null) { + public void recordRowChange(String table, SinkProto.OperationType operation, int rows) + { + if (enabled && rowChangeCounter != null) + { tableChangeCounter.labels(table).inc(rows); rowChangeCounter.labels(table, operation.toString()).inc(rows); } } - public void recordTransaction() { - if (enabled && transactionCounter != null) { + public void recordTransaction() + { + if (enabled && transactionCounter != null) + { transactionCounter.inc(); } } - public Summary.Timer startProcessLatencyTimer() { + public Summary.Timer startProcessLatencyTimer() + { return enabled ? processingLatency.startTimer() : null; } - public Summary.Timer startIndexLatencyTimer() { + public Summary.Timer startIndexLatencyTimer() + { return enabled ? indexServiceLatency.startTimer() : null; } - public Summary.Timer startTransLatencyTimer() { + public Summary.Timer startTransLatencyTimer() + { return enabled ? transServiceLatency.startTimer() : null; } - public Summary.Timer startRetinaLatencyTimer() { + public Summary.Timer startRetinaLatencyTimer() + { return enabled ? retinaServiceLatency.startTimer() : null; } - public Summary.Timer startWriteLatencyTimer() { + public Summary.Timer startWriteLatencyTimer() + { return enabled ? writerLatency.startTimer() : null; } - public void addRawData(double data) { + public void addRawData(double data) + { rawDataThroughputCounter.inc(data); } - public void recordTotalLatency(RowChangeEvent event) { - if(event.getTimeStamp() != 0) { - long recordLatency = System.currentTimeMillis()- event.getTimeStamp(); + public void recordTotalLatency(RowChangeEvent event) + { + if (event.getTimeStamp() != 0) + { + long recordLatency = System.currentTimeMillis() - event.getTimeStamp(); totalLatency.labels(event.getFullTableName(), event.getOp().toString()).observe(recordLatency); } } + + public void recordRowEvent() + { + if (enabled && rowEventCounter != null) + { + rowEventCounter.inc(); + } + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/monitor/MonitorThreadManager.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java similarity index 72% rename from src/main/java/io/pixelsdb/pixels/sink/monitor/MonitorThreadManager.java rename to src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java index 0d25777..4ee0052 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/monitor/MonitorThreadManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.monitor; +package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; @@ -24,39 +24,51 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -public class MonitorThreadManager { + +public class MonitorThreadManager +{ private final List monitors = new CopyOnWriteArrayList<>(); private final ExecutorService executor = Executors.newFixedThreadPool(PixelsSinkConstants.MONITOR_NUM); - public void startMonitor(Runnable monitor) { + public void startMonitor(Runnable monitor) + { monitors.add(monitor); executor.submit(monitor); } - public void shutdown() { + public void shutdown() + { stopMonitors(); shutdownExecutor(); awaitTermination(); } - private void stopMonitors() { - monitors.forEach(monitor -> { - if (monitor instanceof StoppableMonitor) { - ((StoppableMonitor) monitor).stopMonitor(); + private void stopMonitors() + { + monitors.forEach(monitor -> + { + if (monitor instanceof StoppableProcessor) + { + ((StoppableProcessor) monitor).stopProcessor(); } }); } - private void shutdownExecutor() { + private void shutdownExecutor() + { executor.shutdown(); } - private void awaitTermination() { - try { - if (!executor.awaitTermination(10, TimeUnit.SECONDS)) { + private void awaitTermination() + { + try + { + if (!executor.awaitTermination(10, TimeUnit.SECONDS)) + { executor.shutdownNow(); } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { executor.shutdownNow(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java new file mode 100644 index 0000000..a02994c --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java @@ -0,0 +1,84 @@ +package io.pixelsdb.pixels.sink.processor;/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +import io.debezium.embedded.Connect; +import io.debezium.engine.DebeziumEngine; +import io.debezium.engine.RecordChangeEvent; +import io.debezium.engine.format.ChangeEventFormat; +import io.pixelsdb.pixels.sink.PixelsDebeziumConsumer; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * @package: PACKAGE_NAME + * @className: io.pixelsdb.pixels.sink.processor.SinkEngineProcessor + * @author: AntiO2 + * @date: 2025/9/25 09:17 + */ +public class SinkEngineProcessor implements MainProcessor +{ + private final PixelsDebeziumConsumer consumer; + private DebeziumEngine> engine; + private ExecutorService executor; + private volatile boolean running = true; + public SinkEngineProcessor() { + this.consumer = new PixelsDebeziumConsumer(); + } + + public void start() { + Properties debeziumProps = PixelsSinkConfigFactory.getInstance() + .getConfig().extractPropertiesByPrefix("debezium.", true); + + this.engine = DebeziumEngine.create(ChangeEventFormat.of(Connect.class)) + .using(debeziumProps) + .notifying(consumer) + .build(); + + this.executor = Executors.newSingleThreadExecutor(); + this.executor.execute(engine); + } + + @Override + public void stopProcessor() + { + try { + if (engine != null) { + engine.close(); + } + if (executor != null) { + executor.shutdown(); + } + consumer.stopProcessor(); + } catch (Exception e) { + throw new RuntimeException("Failed to stop PixelsSinkEngine", e); + } finally + { + running = false; + } + } + + @Override + public boolean isRunning() + { + return running; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/monitor/SinkMonitor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkKafkaProcessor.java similarity index 67% rename from src/main/java/io/pixelsdb/pixels/sink/monitor/SinkMonitor.java rename to src/main/java/io/pixelsdb/pixels/sink/processor/SinkKafkaProcessor.java index 5e0dd95..48ef519 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/monitor/SinkMonitor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkKafkaProcessor.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.monitor; +package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; @@ -26,49 +26,44 @@ import java.io.IOException; import java.util.Properties; -public class SinkMonitor implements StoppableMonitor { +public class SinkKafkaProcessor implements MainProcessor +{ private MonitorThreadManager manager; private volatile boolean running = true; - private HTTPServer prometheusHttpServer; - public void startSinkMonitor() { + + @Override + public void start() + { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); KafkaPropFactorySelector kafkaPropFactorySelector = new KafkaPropFactorySelector(); Properties transactionKafkaProperties = kafkaPropFactorySelector .getFactory(PixelsSinkConstants.TRANSACTION_KAFKA_PROP_FACTORY) .createKafkaProperties(pixelsSinkConfig); - TransactionMonitor transactionMonitor = new TransactionMonitor(pixelsSinkConfig, transactionKafkaProperties); + TransactionProcessor transactionProcessor = null; // TODO: new TransactionProcessor(); Properties topicKafkaProperties = kafkaPropFactorySelector .getFactory(PixelsSinkConstants.ROW_RECORD_KAFKA_PROP_FACTORY) .createKafkaProperties(pixelsSinkConfig); - TopicMonitor topicMonitor = new TopicMonitor(pixelsSinkConfig, topicKafkaProperties); + TopicProcessor topicMonitor = new TopicProcessor(pixelsSinkConfig, topicKafkaProperties); manager = new MonitorThreadManager(); - manager.startMonitor(transactionMonitor); + manager.startMonitor(transactionProcessor); manager.startMonitor(topicMonitor); - try { - if (pixelsSinkConfig.isMonitorEnabled()) { - this.prometheusHttpServer = new HTTPServer(PixelsSinkConfigFactory.getInstance().getMonitorPort()); - } - } catch (IOException e) { - throw new RuntimeException(e); - } } @Override - public void stopMonitor() { + public void stopProcessor() + { manager.shutdown(); - if (prometheusHttpServer != null) { - prometheusHttpServer.close(); - } - running = false; } - public boolean isRunning() { + @Override + public boolean isRunning() + { return running; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/monitor/StoppableMonitor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java similarity index 84% rename from src/main/java/io/pixelsdb/pixels/sink/monitor/StoppableMonitor.java rename to src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java index 13686ab..61583d8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/monitor/StoppableMonitor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java @@ -15,9 +15,9 @@ * */ -package io.pixelsdb.pixels.sink.monitor; +package io.pixelsdb.pixels.sink.processor; -public interface StoppableMonitor { - - void stopMonitor(); +public interface StoppableProcessor +{ + void stopProcessor(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/consumer/TableConsumerTask.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableMonitor.java similarity index 51% rename from src/main/java/io/pixelsdb/pixels/sink/consumer/TableConsumerTask.java rename to src/main/java/io/pixelsdb/pixels/sink/processor/TableMonitor.java index 8e4164c..9740336 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/consumer/TableConsumerTask.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableMonitor.java @@ -7,26 +7,26 @@ * * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software + * Unless required by applicable law or agreed to in writing, * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, + * either express or implied. * See the License for the specific language governing permissions and * limitations under the License. * */ -package io.pixelsdb.pixels.sink.consumer; +package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.PixelsSinkDefaultConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.WakeupException; import org.slf4j.Logger; @@ -36,21 +36,24 @@ import java.time.Duration; import java.util.Collections; import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; -public class TableConsumerTask implements Runnable { - private static final Logger log = LoggerFactory.getLogger(TableConsumerTask.class); +public class TableMonitor implements Runnable +{ + private static final Logger log = LoggerFactory.getLogger(TableMonitor.class); private static final TransactionCoordinator transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); private final Properties kafkaProperties; private final String topic; private final AtomicBoolean running = new AtomicBoolean(true); private final String tableName; + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); private KafkaConsumer consumer; - private final ExecutorService executor = Executors.newCachedThreadPool(); - public TableConsumerTask(Properties kafkaProperties, String topic) throws IOException { + private Thread processorThread; + + public TableMonitor(Properties kafkaProperties, String topic) throws IOException + { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.kafkaProperties = kafkaProperties; this.topic = topic; @@ -61,54 +64,106 @@ public TableConsumerTask(Properties kafkaProperties, String topic) throws IOExce } @Override - public void run() { - try { + public void run() + { + try + { consumer = new KafkaConsumer<>(kafkaProperties); consumer.subscribe(Collections.singleton(topic)); -// TopicPartition partition = new TopicPartition(topic, 0); -// consumer.poll(Duration.ofSeconds(1)); -// consumer.seek(partition, 0); + processorThread = new Thread(this::processLoop, "processor-" + tableName); + processorThread.start(); - while (running.get()) { - try { + while (running.get()) + { + try + { ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (!records.isEmpty()) { - log.debug("{} Consumer poll returned {} records", tableName, records.count()); - records.forEach(record -> { - executor.execute(() -> { - RowChangeEvent event = record.value(); - transactionCoordinator.processRowEvent(event); - }); + if (!records.isEmpty()) + { + log.info("{} Consumer poll returned {} records", tableName, records.count()); + records.forEach(record -> + { + if (record.value() == null) + { + return; + } + try + { + eventQueue.put(record.value()); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } }); } - } catch (InterruptException ignored) { + } catch (InterruptException ignored) + { } } - } catch (WakeupException e) { - // shutdown normally + } catch (WakeupException e) + { log.info("Consumer wakeup triggered for {}", tableName); - } catch (Exception e) { + } catch (Exception e) + { e.printStackTrace(); log.info("Exception: {}", e.getMessage()); - } finally { - if (consumer != null) { + } finally + { + if (consumer != null) + { consumer.close(Duration.ofSeconds(5)); log.info("Kafka consumer closed for {}", tableName); } + if (processorThread != null) + { + processorThread.interrupt(); + try + { + processorThread.join(); + } catch (InterruptedException ignored) + { + } + } + } + } + + private void processLoop() + { + while (running.get() || !eventQueue.isEmpty()) + { + try + { + RowChangeEvent event = eventQueue.take(); + // log.info("Event received: {}", event); + try + { + transactionCoordinator.processRowEvent(event); + } catch (SinkException e) + { + throw new RuntimeException(e); + } + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } } + log.info("Processor thread exited for {}", tableName); } - public void shutdown() { + public void shutdown() + { running.set(false); log.info("Shutting down consumer for table: {}", tableName); - if (consumer != null) { + if (consumer != null) + { consumer.wakeup(); } } - private String extractTableName(String topic) { + private String extractTableName(String topic) + { String[] parts = topic.split("\\."); return parts[parts.length - 1]; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java new file mode 100644 index 0000000..bc58ceb --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -0,0 +1,89 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.processor; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; +import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.event.TableEventProvider; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * @package: io.pixelsdb.pixels.sink.processor + * @className: TableProcessor + * @author: AntiO2 + * @date: 2025/9/26 11:01 + */ +public class TableProcessor implements StoppableProcessor, Runnable +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TableProcessor.class); + private final AtomicBoolean running = new AtomicBoolean(true); + private Thread processorThread; + private final TransactionCoordinator transactionCoordinator; + private final TableEventProvider tableEventProvider; + private final SchemaTableName schemaTableName; + public TableProcessor(TableEventProvider tableEventProvider, SchemaTableName schemaTableName) + { + this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); + this.tableEventProvider = tableEventProvider; + this.schemaTableName = schemaTableName; + } + + @Override + public void run() + { + processorThread = new Thread(this::processLoop, "processor-table" + schemaTableName.getTableName()); + processorThread.start(); + } + + private void processLoop() + { + while (running.get()) + { + try + { + RowChangeEvent event = tableEventProvider.getSourceEventQueue().take(); + try + { + transactionCoordinator.processRowEvent(event); + } catch (SinkException e) + { + throw new RuntimeException(e); + } + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + } + LOGGER.info("Processor thread exited for {}", schemaTableName.toString()); + } + + @Override + public void stopProcessor() + { + LOGGER.info("Stopping transaction monitor"); + running.set(false); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/monitor/TopicMonitor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java similarity index 69% rename from src/main/java/io/pixelsdb/pixels/sink/monitor/TopicMonitor.java rename to src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java index 7f5eb73..d72d10d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/monitor/TopicMonitor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java @@ -15,10 +15,10 @@ * */ -package io.pixelsdb.pixels.sink.monitor; +package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.consumer.TableConsumerTask; +import io.pixelsdb.pixels.sink.event.TablePipelineManager; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ListTopicsResult; @@ -32,51 +32,55 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -public class TopicMonitor extends Thread implements StoppableMonitor { +public class TopicProcessor extends TablePipelineManager implements StoppableProcessor, Runnable +{ - private static final Logger log = LoggerFactory.getLogger(TopicMonitor.class); + private static final Logger log = LoggerFactory.getLogger(TopicProcessor.class); private final Properties kafkaProperties; private final PixelsSinkConfig pixelsSinkConfig; private final String[] includeTables; private final Set subscribedTopics = ConcurrentHashMap.newKeySet(); private final String bootstrapServers; - private final ExecutorService executorService; private final String baseTopic; private final AtomicBoolean running = new AtomicBoolean(true); + private final Map activeTasks = new ConcurrentHashMap<>(); // track row event consumer private AdminClient adminClient; private Timer timer; - private final Map activeTasks = new ConcurrentHashMap<>(); // track row event consumer - - public TopicMonitor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) { + public TopicProcessor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) + { this.pixelsSinkConfig = pixelsSinkConfig; this.kafkaProperties = kafkaProperties; this.baseTopic = pixelsSinkConfig.getTopicPrefix() + "." + pixelsSinkConfig.getCaptureDatabase(); this.includeTables = pixelsSinkConfig.getIncludeTables(); this.bootstrapServers = pixelsSinkConfig.getBootstrapServers(); - this.executorService = Executors.newCachedThreadPool(); } - private static Set filterTopics(Set topics, String prefix) { + private static Set filterTopics(Set topics, String prefix) + { return topics.stream() .filter(t -> t.startsWith(prefix)) .collect(Collectors.toSet()); } @Override - public void run() { - try { + public void run() + { + try + { initializeResources(); startMonitoringCycle(); - } finally { + } finally + { cleanupResources(); log.info("Topic monitor stopped"); } } @Override - public void stopMonitor() { + public void stopProcessor() + { log.info("Initiating topic monitor shutdown..."); running.set(false); interruptMonitoring(); @@ -84,43 +88,55 @@ public void stopMonitor() { awaitTermination(); } - private void shutdownConsumerTasks() { + private void shutdownConsumerTasks() + { log.info("Shutting down {} active consumer tasks", activeTasks.size()); - activeTasks.forEach((topic, task) -> { + activeTasks.forEach((topic, task) -> + { log.info("Stopping consumer for topic: {}", topic); task.shutdown(); }); activeTasks.clear(); } - private void awaitTermination() { - try { - if (executorService != null && !executorService.awaitTermination(30, TimeUnit.SECONDS)) { + private void awaitTermination() + { + try + { + if (executorService != null && !executorService.awaitTermination(30, TimeUnit.SECONDS)) + { log.warn("Forcing shutdown of remaining tasks"); executorService.shutdownNow(); } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } } - private void initializeResources() { + private void initializeResources() + { Properties props = new Properties(); props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); this.adminClient = AdminClient.create(props); - this.timer = new Timer("TopicMonitor-Timer", true); + this.timer = new Timer("TopicProcessor-Timer", true); log.info("Started topic monitor for base topic: {}", baseTopic); } - private void startMonitoringCycle() { + private void startMonitoringCycle() + { String topicPrefix = baseTopic + "."; timer.scheduleAtFixedRate(new TopicMonitorTask(), 0, 5000); - while (running.get()) { - try { + while (running.get()) + { + try + { TimeUnit.SECONDS.sleep(1); - } catch (InterruptedException e) { - if (running.get()) { + } catch (InterruptedException e) + { + if (running.get()) + { log.warn("Monitoring thread interrupted unexpectedly", e); } Thread.currentThread().interrupt(); @@ -128,105 +144,133 @@ private void startMonitoringCycle() { } } - private void interruptMonitoring() { - if (timer != null) { + private void interruptMonitoring() + { + if (timer != null) + { timer.cancel(); timer.purge(); } - if (adminClient != null) { + if (adminClient != null) + { adminClient.close(Duration.ofSeconds(5)); } shutdownExecutorService(); - this.interrupt(); } - private void shutdownExecutorService() { + private void shutdownExecutorService() + { executorService.shutdown(); - try { - if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) { + try + { + if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) + { executorService.shutdownNow(); } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { executorService.shutdownNow(); Thread.currentThread().interrupt(); } } - private void cleanupResources() { - try { - if (adminClient != null) { + private void cleanupResources() + { + try + { + if (adminClient != null) + { adminClient.close(Duration.ofSeconds(5)); } - } catch (Exception e) { + } catch (Exception e) + { log.warn("Error closing admin client", e); } } - private Set detectNewTopics(Set currentTopics) { + private Set detectNewTopics(Set currentTopics) + { return currentTopics.stream() .filter(t -> !subscribedTopics.contains(t)) .collect(Collectors.toSet()); } - private String extractTableName(String topic) { + private String extractTableName(String topic) + { int lastDotIndex = topic.lastIndexOf('.'); return lastDotIndex != -1 ? topic.substring(lastDotIndex + 1) : topic; } - private void launchConsumerTask(String topic) { - try { - TableConsumerTask task = new TableConsumerTask(kafkaProperties, topic); + private void launchConsumerTask(String topic) + { + try + { + TableMonitor task = new TableMonitor(kafkaProperties, topic); executorService.submit(task); - } catch (Exception e) { + } catch (Exception e) + { log.error("Failed to start consumer for topic {}: {}", topic, e.getMessage()); } } - private class TopicMonitorTask extends TimerTask { + private class TopicMonitorTask extends TimerTask + { @Override - public void run() { - if (!running.get()) { + public void run() + { + if (!running.get()) + { cancel(); return; } - try { + try + { processTopicChanges(); - } catch (Exception e) { + } catch (Exception e) + { e.printStackTrace(); log.error("Error processing topic changes: {}", e.getMessage()); } } - private void processTopicChanges() { - try { + private void processTopicChanges() + { + try + { ListTopicsResult listTopicsResult = adminClient.listTopics(); Set currentTopics = listTopicsResult.names().get(5, TimeUnit.SECONDS); Set filteredTopics = filterTopics(currentTopics, baseTopic + "."); Set newTopics = detectNewTopics(filteredTopics); handleNewTopics(newTopics); - } catch (TimeoutException | ExecutionException | InterruptedException ignored) { + } catch (TimeoutException | ExecutionException | InterruptedException ignored) + { } } - private void handleNewTopics(Set newTopics) { + private void handleNewTopics(Set newTopics) + { newTopics.stream() .filter(this::shouldProcessTable) - .forEach(topic -> { - try { - TableConsumerTask task = new TableConsumerTask(kafkaProperties, topic); + .forEach(topic -> + { + try + { + TableMonitor task = new TableMonitor(kafkaProperties, topic); executorService.submit(task); activeTasks.put(topic, task); subscribedTopics.add(topic); - } catch (IOException e) { + } catch (IOException e) + { log.error("Failed to create consumer for {}: {}", topic, e.getMessage()); } }); } - private boolean shouldProcessTable(String topic) { + private boolean shouldProcessTable(String topic) + { String tableName = extractTableName(topic); return includeTables.length == 0 || Arrays.stream(includeTables).anyMatch(t -> t.equals(tableName)); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java new file mode 100644 index 0000000..1f16bcb --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -0,0 +1,79 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, + * either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.processor; + +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; +import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.event.TransactionEventProvider; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +public class TransactionProcessor implements Runnable, StoppableProcessor +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionProcessor.class); + private final TransactionCoordinator transactionCoordinator; + private final AtomicBoolean running = new AtomicBoolean(true); + private final TransactionEventProvider transactionEventProvider; + + public TransactionProcessor(TransactionEventProvider transactionEventProvider) + { + this.transactionEventProvider = transactionEventProvider; + this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); + } + + @Override + public void run() + { + while (running.get()) + { + try + { + SinkProto.TransactionMetadata transaction = transactionEventProvider.getEventQueue().take(); + try + { + LOGGER.info("Processing transaction event: {}", transaction.getId()); + transactionCoordinator.processTransactionEvent(transaction); + } catch (SinkException e) + { + throw new RuntimeException(e); + } + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + } + LOGGER.info("Processor thread exited for transaction"); + } + + @Override + public void stopProcessor() + { + LOGGER.info("Stopping transaction monitor"); + running.set(false); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java index 536bed8..85dd1a4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java @@ -17,6 +17,7 @@ package io.pixelsdb.pixels.sink.sink; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkDefaultConfig; @@ -42,12 +43,12 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -public class CsvWriter implements PixelsSinkWriter { +public class CsvWriter implements PixelsSinkWriter +{ private static final Logger log = LoggerFactory.getLogger(CsvWriter.class); - private Long recordCnt = 0L; - @Getter private static final PixelsSinkMode pixelsSinkMode = PixelsSinkMode.CSV; + private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); private final ReentrantLock lock = new ReentrantLock(); private final ConcurrentMap tableWriters = new ConcurrentHashMap<>(); private final ScheduledExecutorService flushScheduler; @@ -57,16 +58,17 @@ public class CsvWriter implements PixelsSinkWriter { private final ReentrantLock globalLock = new ReentrantLock(); private final ReentrantLock writeLock = new ReentrantLock(true); private final AtomicInteger writeCounter = new AtomicInteger(0); - - private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); private final String CSV_DELIMITER = "|"; + private final Long recordCnt = 0L; - public CsvWriter() throws IOException { + public CsvWriter() throws IOException + { this.databaseName = config.getCaptureDatabase(); this.baseOutputPath = Paths.get(config.getCsvSinkPath(), databaseName); this.enableHeader = config.isSinkCsvEnableHeader(); - if (!Files.exists(baseOutputPath)) { + if (!Files.exists(baseOutputPath)) + { Files.createDirectories(baseOutputPath); } @@ -77,104 +79,145 @@ public CsvWriter() throws IOException { } @Override - public void flush() { + public void flush() + { writeLock.lock(); - try { - for (FileChannel channel : tableWriters.values()) { - try { + try + { + for (FileChannel channel : tableWriters.values()) + { + try + { channel.force(true); - } catch (IOException e) { + } catch (IOException e) + { log.error("Failed to flush channel {}", channel, e); } } writeCounter.set(0); - } finally { + } finally + { writeLock.unlock(); } } @Override - public boolean write(RowChangeEvent event) { + public boolean write(RowChangeEvent event) + { final String tableName = event.getTable(); - if (event.getOp() == SinkProto.OperationType.DELETE) { + if (event.getOp() == SinkProto.OperationType.DELETE) + { return true; } // Map message = event.getAfterData(); Map message = null; writeLock.lock(); - try { + try + { FileChannel channel = getOrCreateChannel(event); String csvLine = convertToCSV(message); ByteBuffer buffer = ByteBuffer.wrap((csvLine + "\n").getBytes(StandardCharsets.UTF_8)); - while (buffer.hasRemaining()) { + while (buffer.hasRemaining()) + { channel.write(buffer); } - if (writeCounter.incrementAndGet() % PixelsSinkDefaultConfig.SINK_CSV_RECORD_FLUSH == 0) { + if (writeCounter.incrementAndGet() % PixelsSinkDefaultConfig.SINK_CSV_RECORD_FLUSH == 0) + { channel.force(false); } return true; - } catch (IOException e) { + } catch (IOException e) + { log.error("Write failed for table {}: {}", tableName, e.getMessage()); return false; - } finally { + } finally + { writeLock.unlock(); } } - private FileChannel getOrCreateChannel(RowChangeEvent event) throws IOException { + @Override + public boolean writeTrans(String schemaName, List tableUpdateData, long timestamp) + { + throw new UnsupportedOperationException("CSV Writer doesn't implement batch write trans"); + } + + @Override + public boolean writeBatch(String schemaName, List tableUpdateData) + { + throw new UnsupportedOperationException("CSV Writer doesn't implement batch write batch"); + } + + private FileChannel getOrCreateChannel(RowChangeEvent event) throws IOException + { String tableName = event.getTable(); - return tableWriters.computeIfAbsent(tableName, key -> { - try { + return tableWriters.computeIfAbsent(tableName, key -> + { + try + { Path tablePath = baseOutputPath.resolve(tableName + ".csv"); FileChannel channel = FileChannel.open(tablePath, StandardOpenOption.CREATE, StandardOpenOption.APPEND, StandardOpenOption.WRITE); - if (channel.size() == 0) { + if (channel.size() == 0) + { String header = String.join(CSV_DELIMITER, getHeaderFields(event)); channel.write(ByteBuffer.wrap((header + "\n").getBytes())); } return channel; - } catch (IOException e) { + } catch (IOException e) + { throw new UncheckedIOException("Failed to create channel for " + tableName, e); } }); } - private String convertToCSV(Map message) { + private String convertToCSV(Map message) + { return message.values().stream() - .map(obj -> { + .map(obj -> + { if (obj == null) return ""; return obj.toString(); }) .collect(Collectors.joining(CSV_DELIMITER)); } - private List getHeaderFields(RowChangeEvent event) { + private List getHeaderFields(RowChangeEvent event) + { + return event.getSchema().getFieldNames(); } @Override - public void close() throws IOException { + public void close() throws IOException + { flushScheduler.shutdown(); - try { - if (!flushScheduler.awaitTermination(5, TimeUnit.SECONDS)) { + try + { + if (!flushScheduler.awaitTermination(5, TimeUnit.SECONDS)) + { flushScheduler.shutdownNow(); } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } globalLock.lock(); - try { - for (FileChannel channel : tableWriters.values()) { + try + { + for (FileChannel channel : tableWriters.values()) + { channel.close(); } tableWriters.clear(); - } finally { + } finally + { globalLock.unlock(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java index d75e36f..10bd82b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java @@ -18,13 +18,17 @@ package io.pixelsdb.pixels.sink.sink; -public enum PixelsSinkMode { +public enum PixelsSinkMode +{ CSV, RETINA; - public static PixelsSinkMode fromValue(String value) { - for (PixelsSinkMode mode : values()) { - if (mode.name().equalsIgnoreCase(value)) { + public static PixelsSinkMode fromValue(String value) + { + for (PixelsSinkMode mode : values()) + { + if (mode.name().equalsIgnoreCase(value)) + { return mode; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java index a71a345..d04acb5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java @@ -17,14 +17,19 @@ package io.pixelsdb.pixels.sink.sink; +import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import java.io.Closeable; +import java.util.List; -public interface PixelsSinkWriter extends Closeable { +public interface PixelsSinkWriter extends Closeable +{ void flush(); boolean write(RowChangeEvent rowChangeEvent); - // boolean write(RowChangeEvent rowChangeEvent, ByteBuffer byteBuffer); + boolean writeTrans(String schemaName, List tableUpdateData, long timestamp); + + boolean writeBatch(String schemaName, List tableUpdateData); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java index e2f2d05..4721b07 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java @@ -22,11 +22,14 @@ import java.io.IOException; -public class PixelsSinkWriterFactory { +public class PixelsSinkWriterFactory +{ private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - static public PixelsSinkWriter getWriter() throws IOException { - switch (config.getPixelsSinkMode()) { + static public PixelsSinkWriter getWriter() throws IOException + { + switch (config.getPixelsSinkMode()) + { case CSV: return new CsvWriter(); case RETINA: diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java index 5e1b9a1..7a678b3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java @@ -17,175 +17,209 @@ package io.pixelsdb.pixels.sink.sink; -import io.grpc.ManagedChannel; -import io.grpc.ManagedChannelBuilder; -import io.grpc.StatusRuntimeException; -import io.pixelsdb.pixels.common.index.IndexService; -import io.pixelsdb.pixels.index.IndexProto; +import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.retina.RetinaProto; -import io.pixelsdb.pixels.retina.RetinaWorkerServiceGrpc; -import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.concurrent.TransactionMode; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.monitor.MetricsFacade; -import io.pixelsdb.pixels.sink.util.LatencySimulator; -import io.prometheus.client.Summary; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.concurrent.TimeUnit; +import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; -public class RetinaWriter implements PixelsSinkWriter { +public class RetinaWriter implements PixelsSinkWriter +{ private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriter.class); @Getter private static final PixelsSinkMode pixelsSinkMode = PixelsSinkMode.RETINA; - private static final IndexService indexService = IndexService.Instance(); - - private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + // private static final IndexService indexService = IndexService.Instance(); private final AtomicBoolean isClosed = new AtomicBoolean(false); - // TODO(LiZinuo): 使用RetinaService替换 - final ManagedChannel channel; - final RetinaWorkerServiceGrpc.RetinaWorkerServiceBlockingStub blockingStub; - + private final RetinaService retinaService = RetinaService.Instance(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - - RetinaWriter() { - if (config.isRpcEnable()) { - // TODO(LiZinuo): 使用RetinaService替换 - this.channel = ManagedChannelBuilder.forAddress( - config.getSinkRemoteHost(), - config.getRemotePort() - ) - .usePlaintext() - .build(); - this.blockingStub = RetinaWorkerServiceGrpc.newBlockingStub(channel); - } else { - channel = null; - blockingStub = null; + private RetinaService.StreamHandler retinaStream = null; + public RetinaWriter() + { + if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) + { + retinaStream = retinaService.startUpdateStream(); + } else + { + retinaStream = null; } } @Override - public void flush() { + public void flush() + { } @Override - public boolean write(RowChangeEvent event) { - if (isClosed.get()) { + public boolean write(RowChangeEvent event) + { + if (isClosed.get()) + { LOGGER.warn("Attempted to write to closed writer"); return false; } - try { - if (config.isRpcEnable()) { - switch (event.getOp()) { - case INSERT: - case SNAPSHOT: - return sendInsertRequest(event); - case UPDATE: - return sendUpdateRequest(event); - case DELETE: - return sendDeleteRequest(event); - case UNRECOGNIZED: - break; - } - } else { - if (event.getOp() != SinkProto.OperationType.INSERT && event.getOp() != SinkProto.OperationType.SNAPSHOT) { - Summary.Timer indexLatencyTimer = metricsFacade.startIndexLatencyTimer(); - LatencySimulator.smartDelay(); // Mock Look Up Index - indexLatencyTimer.close(); - } - Summary.Timer retinaLatencyTimer = metricsFacade.startRetinaLatencyTimer(); - LatencySimulator.smartDelay(); // Call Retina - retinaLatencyTimer.close(); - return true; + try + { + switch (event.getOp()) + { + case INSERT: + case SNAPSHOT: + return sendInsertRequest(event); + case UPDATE: + return sendUpdateRequest(event); + case DELETE: + return sendDeleteRequest(event); + case UNRECOGNIZED: + break; } - - } catch (StatusRuntimeException e) { - LOGGER.error("gRPC write failed for event {}: {}", event.getTransaction().getId(), e.getStatus()); + } catch (RetinaException e) + { + LOGGER.error("Retina write failed for event {}", event); return false; - } finally { - } // TODO: error handle return false; } - private boolean sendInsertRequest(RowChangeEvent event) { - // TODO 这里需要RetinaService提供包装接口 - RetinaProto.InsertRecordResponse insertRecordResponse = blockingStub.insertRecord(getInsertRecordRequest(event)); - return insertRecordResponse.getHeader().getErrorCode() == 0; - } - - private RetinaProto.InsertRecordRequest getInsertRecordRequest(RowChangeEvent event) { - // Step1. Serialize Row Data - RetinaProto.InsertRecordRequest.Builder builder = RetinaProto.InsertRecordRequest.newBuilder(); - RetinaProto.RowValue.Builder rowValueBuilder = builder.getRowBuilder(); - event.getAfterData().getValuesList().forEach(value -> - builder.setRow(rowValueBuilder.addValues(value.getValue()))); - - // Step2. Build Insert Request - return builder - .setSchema(event.getDb()) - .setTable(event.getTable()) - .setRow(rowValueBuilder.build()) - .setTimestamp(event.getTimeStamp()) - .setTransInfo(getTransinfo(event)) - .build(); + @Override + public boolean writeTrans(String schemaName, List tableUpdateData, long timestamp) + { + if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) + { + try + { + LOGGER.debug("Retina Writer update record {}, TS: {}", schemaName, timestamp); + retinaService.updateRecord(schemaName, tableUpdateData); + } catch (RetinaException e) + { + e.printStackTrace(); + return false; + } + } else + { + retinaStream.updateRecord(schemaName, tableUpdateData); + } + return true; } - private boolean sendDeleteRequest(RowChangeEvent event) { - RetinaProto.DeleteRecordResponse deleteRecordResponse = blockingStub.deleteRecord(getDeleteRecordRequest(event)); - return deleteRecordResponse.getHeader().getErrorCode() == 0; + @Override + public boolean writeBatch(String schemaName, List tableUpdateData) + { + if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) + { + try + { + retinaService.updateRecord(schemaName, tableUpdateData); + } catch (RetinaException e) + { + e.printStackTrace(); + return false; + } + } else + { + retinaStream.updateRecord(schemaName, tableUpdateData); + } + return true; } - @Override - public void close() throws IOException { - if (isClosed.compareAndSet(false, true)) { - try { - channel.shutdown(); - if (!channel.awaitTermination(5, TimeUnit.SECONDS)) { - channel.shutdownNow(); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException("Channel shutdown interrupted", e); + public CompletableFuture writeBatchAsync + (String schemaName, List tableUpdateData) + { + if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) + { + try + { + retinaService.updateRecord(schemaName, tableUpdateData); + } catch (RetinaException e) + { + e.printStackTrace(); } + return null; + } else + { + return retinaStream.updateRecord(schemaName, tableUpdateData); } } - private RetinaProto.DeleteRecordRequest getDeleteRecordRequest(RowChangeEvent event) { - // Step1. Look up unique index to find row location - IndexProto.RowLocation rowLocation = indexService.lookupUniqueIndex(event.getIndexKey()); - - // Step2. Build Delete Request - RetinaProto.DeleteRecordRequest.Builder builder = RetinaProto.DeleteRecordRequest.newBuilder(); - return builder - .setRow(builder.getRowBuilder().setRgRowId(rowLocation.getRgRowId()).setFileId(rowLocation.getFileId()).setRgId(rowLocation.getRgId())) - .setTimestamp(event.getTimeStamp()) - .setTransInfo(getTransinfo(event)) - .build(); + + @Deprecated + private boolean sendInsertRequest(RowChangeEvent event) throws RetinaException + { + // Insert retina + // boolean retinaServiceResult = retinaService.insertRecord(event.getSchemaName(), event.getTable(), event.getAfterData(), event.getTimeStamp()); + + return false; } - private boolean sendUpdateRequest(RowChangeEvent event) { - // Delete & Insert - RetinaProto.DeleteRecordResponse deleteRecordResponse = blockingStub.deleteRecord(getDeleteRecordRequest(event)); - if (deleteRecordResponse.getHeader().getErrorCode() != 0) { - return false; + private boolean sendDeleteRequest(RowChangeEvent event) + { + return false; +// RetinaProto.DeleteRecordResponse deleteRecordResponse = blockingStub.deleteRecord(getDeleteRecordRequest(event)); +// return deleteRecordResponse.getHeader().getErrorCode() == 0; + } + + @Override + public void close() throws IOException + { + if (isClosed.compareAndSet(false, true)) + { +// try { +// channel.shutdown(); +// if (!channel.awaitTermination(5, TimeUnit.SECONDS)) { +// channel.shutdownNow(); +// } +// } catch (InterruptedException e) { +// Thread.currentThread().interrupt(); +// throw new IOException("Channel shutdown interrupted", e); +// } } - RetinaProto.InsertRecordResponse insertRecordResponse = blockingStub.insertRecord(getInsertRecordRequest(event)); - return insertRecordResponse.getHeader().getErrorCode() == 0; + if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) + { + retinaStream.close(); + } + } + + private boolean sendUpdateRequest(RowChangeEvent event) + { + // Delete & Insert +// RetinaProto.DeleteRecordResponse deleteRecordResponse = blockingStub.deleteRecord(getDeleteRecordRequest(event)); +// if (deleteRecordResponse.getHeader().getErrorCode() != 0) { +// return false; +// } +// +// RetinaProto.InsertRecordResponse insertRecordResponse = blockingStub.insertRecord(getInsertRecordRequest(event)); +// return insertRecordResponse.getHeader().getErrorCode() == 0; + return false; } - private RetinaProto.TransInfo getTransinfo(RowChangeEvent event) { - return RetinaProto.TransInfo.newBuilder() - .setOrder(event.getTransaction().getTotalOrder()) - .setTransId(event.getTransaction().getId().hashCode()).build(); + public enum RetinaWriteMode + { + STREAM, + STUB; + + public static RetinaWriteMode fromValue(String value) + { + for (RetinaWriteMode mode : values()) + { + if (mode.name().equalsIgnoreCase(value)) + { + return mode; + } + } + throw new RuntimeException(String.format("Can't convert %s to sink type", value)); + } } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java new file mode 100644 index 0000000..71ad4f9 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java @@ -0,0 +1,189 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.sink; + + +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.concurrent.SinkContext; +import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import lombok.Getter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.locks.ReentrantLock; + +/** + * @package: io.pixelsdb.pixels.sink.sink + * @className: TableCrossTxWriter + * @author: AntiO2 + * @date: 2025/9/27 09:36 + */ +public class TableCrossTxWriter extends TableWriter +{ + @Getter + private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); + private final int flushBatchSize; + private final ReentrantLock writeLock = new ReentrantLock(); + + public TableCrossTxWriter(String t) throws IOException + { + super(t); + flushBatchSize = config.getFlushBatchSize(); + } + + /** + * Flush any buffered events for the current transaction. + */ + public void flush() + { + List batch; + lock.lock(); + try + { + if (buffer.isEmpty()) + { + return; + } + // Swap buffers quickly under lock + batch = buffer; + buffer = new LinkedList<>(); + } finally + { + lock.unlock(); + } + + writeLock.lock(); + try + { + + String txId = null; + String schemaName = null; + List smallBatch = null; + List txIds = new ArrayList<>(); + List fullTableName = new ArrayList<>(); + List tableUpdateData = new LinkedList<>(); + List tableUpdateCount = new ArrayList<>(); + for(RowChangeEvent event : batch) + { + String currTxId = event.getTransaction().getId(); + if(!currTxId.equals(txId)) + { + if(smallBatch != null && !smallBatch.isEmpty()) + { + tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch)); + tableUpdateCount.add(smallBatch.size()); + } + txIds.add(currTxId); + fullTableName.add(event.getFullTableName()); + txId = currTxId; + smallBatch = new LinkedList<>(); + } + smallBatch.add(event); + } + + if(smallBatch != null) + { + tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch)); + tableUpdateCount.add(smallBatch.size()); + } + + CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); + + updateRecordResponseCompletableFuture.thenAccept( + resp -> { + for(int i = 0; i < txIds.size(); i++) + { + String writeTxId = txIds.get(i); + SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(writeTxId); + sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); + } + } + ); + } finally + { + writeLock.unlock(); + } + } + + private RetinaProto.TableUpdateData buildTableUpdateDataFromBatch(String txId, List smallBatch) + { + SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(txId); + try + { + sinkContext.getLock().lock(); + while (sinkContext.getPixelsTransCtx() == null) + { + LOGGER.warn("Wait for tx to begin trans: {}", txId); // CODE SHOULD NOT REACH HERE + sinkContext.getCond().await(); + } + } catch (InterruptedException e) + { + throw new RuntimeException(e); + } finally + { + sinkContext.getLock().unlock(); + } + RowChangeEvent event1 = smallBatch.get(0); + + RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder() + .setTimestamp(sinkContext.getTimestamp()) + .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) + .setTableName(tableName); + try + { + for (RowChangeEvent smallEvent : smallBatch) + { + addUpdateData(smallEvent, builder); + } + } catch (SinkException e) + { + throw new RuntimeException("Flush failed for table " + tableName, e); + } + return builder.build(); + } + + @Override + protected boolean needFlush() + { + return buffer.size() >= flushBatchSize; + } + + public static TableWriter getTableWriter(String tableName) + { + return WRITER_REGISTRY.computeIfAbsent(tableName, t -> + { + try + { + return new TableCrossTxWriter(t); + } catch (IOException e) + { + throw new RuntimeException(e); + } + }); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java new file mode 100644 index 0000000..47fcf2b --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java @@ -0,0 +1,137 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.pixelsdb.pixels.sink.sink; + +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.concurrent.SinkContext; +import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import lombok.Getter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.locks.ReentrantLock; + +public class TableSingleTxWriter extends TableWriter +{ + @Getter + private final Logger LOGGER = LoggerFactory.getLogger(TableSingleTxWriter.class); + private static final long TX_TIMEOUT_MS = 3000; + public TableSingleTxWriter(String tableName) throws IOException + { + super(tableName); + } + + public static TableWriter getTableWriter(String tableName) + { + return WRITER_REGISTRY.computeIfAbsent(tableName, t -> + { + try + { + return new TableSingleTxWriter(t); + } catch (IOException e) + { + throw new RuntimeException(e); + } + }); + } + + /** + * Flush any buffered events for the current transaction. + */ + public void flush() + { + List batch; + String txId; + RetinaProto.TableUpdateData.Builder toBuild; + SinkContext sinkContext = null; + lock.lock(); + try + { + if (buffer.isEmpty() || currentTxId == null) + { + return; + } + txId = currentTxId; + currentTxId = null; + + sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(txId); + sinkContext.getLock().lock(); + try + { + while (sinkContext.getPixelsTransCtx() == null) + { + LOGGER.warn("Wait for prev tx to begin trans: {}", txId); + sinkContext.getCond().await(); + } + } finally + { + sinkContext.getLock().unlock(); + } + + // Swap buffers quickly under lock + batch = buffer; + buffer = new ArrayList<>(); + } catch (InterruptedException e) + { + throw new RuntimeException(e); + } finally + { + lock.unlock(); + } + + RowChangeEvent event1 = batch.get(0); + + RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder() + .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) + .setTableName(tableName); + + + try + { + for (RowChangeEvent event : batch) + { + addUpdateData(event, builder); + } + List tableUpdateData = List.of(builder.build()); + delegate.writeTrans(event1.getSchemaName(), tableUpdateData, sinkContext.getTimestamp()); + sinkContext.updateCounter(fullTableName, batch.size()); + // ---- Outside lock: build proto and write ---- + LOGGER.info("Flushing {} events for table {} txId={}", batch.size(), fullTableName, txId); + } catch (SinkException e) + { + throw new RuntimeException("Flush failed for table " + tableName, e); + } + } + + @Override + protected boolean needFlush() + { + if (currentTxId == null || !currentTxId.equals(txId)) + { + return !buffer.isEmpty(); + } + return false; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java new file mode 100644 index 0000000..48861b6 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java @@ -0,0 +1,173 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.sink; + + +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.concurrent.SinkContext; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.slf4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.locks.ReentrantLock; + +/** + * @package: io.pixelsdb.pixels.sink.sink + * @className: TableWriter + * @author: AntiO2 + * @date: 2025/9/27 09:58 + */ +public abstract class TableWriter +{ + + // TODO(AntiO2): 这里放弃掉底层writer的多种实现了。 + protected final RetinaWriter delegate; // physical writer + protected final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + protected final ReentrantLock lock = new ReentrantLock(); + protected final String tableName; + // Shared state (protected by lock) + protected List buffer = new LinkedList<>(); + protected volatile String currentTxId = null; + protected String txId = null; + protected ScheduledFuture flushTask = null; + protected String fullTableName; + protected PixelsSinkConfig config; + protected TableWriter(String tableName) throws IOException + { + this.config = PixelsSinkConfigFactory.getInstance(); + this.tableName = tableName; + this.delegate = new RetinaWriter(); + this.flushInterval = config.getFlushIntervalMs(); + } + + protected static final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); + protected abstract Logger getLOGGER(); + protected final long flushInterval; + + + public boolean write(RowChangeEvent event, SinkContext ctx) + { + try + { + lock.lock(); + try + { + txId = ctx.getSourceTxId(); + // If this is a new transaction, flush the old one + if(needFlush()) + { + if (flushTask != null) + { + flushTask.cancel(false); + } + flush(); + + } + currentTxId = txId; + if (fullTableName == null) + { + fullTableName = event.getFullTableName(); + } + buffer.add(event); + + // Reset scheduled flush: cancel old one and reschedule + if (flushTask != null && !flushTask.isDone()) + { + flushTask.cancel(false); + } + flushTask = scheduler.schedule(() -> + { + try + { + lock.lock(); + try + { + if (txId.equals(currentTxId)) + { + flush(); + } + } finally + { + lock.unlock(); + } + } catch (Exception e) + { + getLOGGER().error("Scheduled flush failed for table {}", tableName, e); + } + }, flushInterval, TimeUnit.MILLISECONDS); + } finally + { + lock.unlock(); + } + return true; + } catch (Exception e) + { + getLOGGER().error("Write failed for table {}", tableName, e); + return false; + } + } + + public abstract void flush(); + + protected abstract boolean needFlush(); + + public void close() + { + scheduler.shutdown(); + try + { + scheduler.awaitTermination(5, TimeUnit.SECONDS); + delegate.close(); + } catch (InterruptedException ignored) + { + } catch (IOException e) + { + throw new RuntimeException(e); + } + } + /** + * Helper: add insert/delete data into proto builder. + */ + public static void addUpdateData(RowChangeEvent rowChangeEvent, + RetinaProto.TableUpdateData.Builder builder) throws SinkException + { + if (rowChangeEvent.hasBeforeData()) + { + RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); + deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); + builder.addDeleteData(deleteDataBuilder); + } + + if (rowChangeEvent.hasAfterData()) + { + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); + insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); + insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); + builder.addInsertData(insertDataBuilder); + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java b/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java new file mode 100644 index 0000000..115d585 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java @@ -0,0 +1,89 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.util; + + +import io.pixelsdb.pixels.core.utils.DatetimeUtils; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Calendar; +import java.util.Date; + +/** + * @package: io.pixelsdb.pixels.sink.util + * @className: DateUtil + * @author: AntiO2 + * @date: 2025/8/21 17:31 + */ +public class DateUtil +{ + + public static Date fromDebeziumDate(int epochDay) + { + Calendar cal = Calendar.getInstance(); + cal.clear(); + cal.set(1970, Calendar.JANUARY, 1); // epoch 起点 + cal.add(Calendar.DAY_OF_MONTH, epochDay); // 加上天数 + return cal.getTime(); + } + + // TIMESTAMP(1), TIMESTAMP(2), TIMESTAMP(3) + public static Date fromDebeziumTimestamp(long epochTs) + { + return new Date(epochTs / 1000); + } + + public static String convertDateToDayString(Date date) + { + // "yyyy-MM-dd HH:mm:ss.SSS" + DateFormat df = new SimpleDateFormat("yyyy-MM-dd"); + String dateToString = df.format(date); + return (dateToString); + } + + public static String convertDateToString(Date date) + { + // "yyyy-MM-dd HH:mm:ss.SSS" + DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + String dateToString = df.format(date); + return (dateToString); + } + + public static String convertTimestampToString(Date date) + { + if (date == null) + { + return null; + } + SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss.SSS"); + return sdf.format(date); + } + + public static String convertDebeziumTimestampToString(long epochTs) + { + LocalDateTime dateTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(epochTs), ZoneId.systemDefault()); + DateTimeFormatter formatter = DatetimeUtils.SQL_LOCAL_DATE_TIME; + return dateTime.format(formatter); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java b/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java index 1a4913d..90e598c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java @@ -23,15 +23,18 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -public class LatencySimulator { +public class LatencySimulator +{ private static final Random RANDOM = ThreadLocalRandom.current(); private static final double longTailProb = 0.05; private static final double longTailScale = 30; private static final double tailVariance = 0.1; private static final double normalVariance = 0.4; - private static long generateLongTailDelay(long baseDelayMs) { - if (RANDOM.nextDouble() < longTailProb) { + private static long generateLongTailDelay(long baseDelayMs) + { + if (RANDOM.nextDouble() < longTailProb) + { double variance = 1 + (RANDOM.nextDouble() * 2 - 1) * tailVariance; return (long) (baseDelayMs * longTailScale * variance); } @@ -39,10 +42,13 @@ private static long generateLongTailDelay(long baseDelayMs) { return (long) (baseDelayMs * (1 + (RANDOM.nextDouble() - 0.5) * normalVariance)); } - public static void smartDelay() { - try { + public static void smartDelay() + { + try + { TimeUnit.MILLISECONDS.sleep(generateLongTailDelay(PixelsSinkConfigFactory.getInstance().getMockRpcDelay())); - } catch (InterruptedException ignored) { + } catch (InterruptedException ignored) + { } } diff --git a/src/main/resources/log4j2.properties b/src/main/resources/log4j2.properties index e282341..7a254cc 100644 --- a/src/main/resources/log4j2.properties +++ b/src/main/resources/log4j2.properties @@ -1,28 +1,31 @@ -status = info -name = pixels-sink - -filter.threshold.type = ThresholdFilter -#filter.threshold.level = info -filter.threshold.level=debug - -appender.console.type = Console -appender.console.name = STDOUT -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n - -appender.rolling.type = File -appender.rolling.name = log -appender.rolling.append = true -appender.rolling.fileName = ${env:PIXELS_HOME}/logs/pixels-sink.log -appender.rolling.layout.type = PatternLayout -appender.rolling.layout.pattern = %-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n - -rootLogger.level = info -rootLogger.appenderRef.stdout.ref = STDOUT -rootLogger.appenderRef.log.ref = log - +status=info +name=pixels-sink +filter.threshold.type=ThresholdFilter +filter.threshold.level=warn +appender.console.type=Console +appender.console.name=STDOUT +appender.console.layout.type=PatternLayout +appender.console.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n +appender.rolling.type=File +appender.rolling.name=log +appender.rolling.append=true +appender.rolling.fileName=${env:PIXELS_HOME}/logs/pixels-sink.log +appender.rolling.layout.type=PatternLayout +appender.rolling.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n +rootLogger.level=info +rootLogger.appenderRef.stdout.ref=STDOUT +rootLogger.appenderRef.log.ref=log logger.transaction.name=io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator logger.transaction.level=info logger.transaction.appenderRef.log.ref=log logger.transaction.appenderRef.stdout.ref=STDOUT -logger.transaction.additivity=false \ No newline at end of file +logger.transaction.additivity=false +logger.grpc.name=io.grpc.netty.shaded.io.grpc.netty.NettyClientHandler +logger.grpc.level=info +logger.grpc.additivity=false +logger.grpc.appenderRef.log.ref=log +logger.grpc.appenderRef.stdout.ref=STDOUT +log4j2.logger.io.grpc.netty.shaded.io.grpc.netty.NettyClientHandler=OFF +logger.netty-shaded.name=io.grpc.netty.shaded.io.netty +logger.netty-shaded.level=info +logger.netty-shaded.additivity=false diff --git a/src/main/resources/logging.properties b/src/main/resources/logging.properties new file mode 100644 index 0000000..2c1fd24 --- /dev/null +++ b/src/main/resources/logging.properties @@ -0,0 +1,3 @@ +.level=INFO +io.grpc.level=INFO +io.grpc.netty.level=INFO diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties new file mode 100644 index 0000000..3d1f5ce --- /dev/null +++ b/src/main/resources/pixels-sink.aws.properties @@ -0,0 +1,74 @@ +# engine | kafka +sink.datasource=engine + +# embedded engine config + + +# Kafka Config +bootstrap.servers=realtime-kafka-2:29092 +group.id=3078 +auto.offset.reset=earliest +key.deserializer=org.apache.kafka.common.serialization.StringDeserializer +#value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer +value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer +# Topic & Database Config +topic.prefix=postgresql.oltp_server +consumer.capture_database=pixels_bench_sf1x +consumer.include_tables= +# Sink Config +sink.mode=retina +sink.csv.path=./data +sink.csv.enable_header=false +## Retina Config +sink.retina.embedded=false +# stub or stream + sink.retina.mode=stream +#sink.retina.mode=stub +sink.remote.host=localhost +sink.remote.port=29422 +sink.batch.size=100 +sink.timeout.ms=5000 +sink.flush.interval.ms=100 +sink.flush.batch.size=200 +sink.max.retries=3 +## Schema Registry +sink.registry.url=http://localhost:8080/apis/registry/v2 +# Transaction Config +transaction.topic.suffix=transaction +#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer +sink.trans.batch.size=100 +## Batch or trans or record +sink.trans.mode=batch +# Sink Metrics +sink.monitor.enable=true +sink.monitor.port=9464 +# Interact with other rpc +sink.rpc.enable=true +sink.rpc.mock.delay=20 + +# debezium +debezium.name=testEngine +debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector +debezium.provide.transaction.metadata=true +debezium.offset.storage=org.apache.kafka.connect.storage.FileOffsetBackingStore +debezium.offset.storage.file.filename=/tmp/offsets.dat +debezium.offset.flush.interval.ms=60000 +debezium.schema.history.internal=io.debezium.storage.file.history.FileSchemaHistory +debezium.schema.history.internal.file.filename=/tmp/schemahistory.dat +debezium.database.hostname=realtime-pg-2 +debezium.database.port=5432 +debezium.database.user=pixels +debezium.database.password=pixels_realtime_crud +debezium.database.dbname=pixels_bench_sf1x +debezium.plugin.name=pgoutput +debezium.database.server.id=1 +debezium.schema.include.list=public +debezium.snapshot.mode=never +debezium.key.converter=org.apache.kafka.connect.json.JsonConverter +debezium.value.converter=org.apache.kafka.connect.json.JsonConverter +debezium.topic.prefix=postgresql.oltp_server +debezium.transforms=topicRouting +debezium.transforms.topicRouting.type=org.apache.kafka.connect.transforms.RegexRouter +debezium.transforms.topicRouting.regex=postgresql\\.oltp_server\\.public\\.(.*) +debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf1x.$1 diff --git a/src/main/resources/pixels-sink.local.properties b/src/main/resources/pixels-sink.local.properties index 1d9ed13..bfb62b5 100644 --- a/src/main/resources/pixels-sink.local.properties +++ b/src/main/resources/pixels-sink.local.properties @@ -1,37 +1,75 @@ +# engine | kafka +sink.datasource=engine + +# embedded engine config + + # Kafka Config bootstrap.servers=localhost:29092 -group.id=2050 +group.id=3107 auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer -value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer - +#value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer +value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config -topic.prefix=oltp_server -consumer.capture_database=pixels_realtime_crud +topic.prefix=postgresql.oltp_server +consumer.capture_database=pixels_bench_sf1x consumer.include_tables= - # Sink Config sink.mode=retina sink.csv.path=./data sink.csv.enable_header=false - +## Retina Config +sink.retina.embedded=false +# stub or stream +sink.retina.mode=stream +#sink.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 sink.batch.size=100 sink.timeout.ms=5000 -sink.flush.interval.ms=5000 +sink.flush.interval.ms=100 +sink.flush.batch.size=100 sink.max.retries=3 - ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 - # Transaction Config transaction.topic.suffix=transaction -transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 +## Batch or trans or record +sink.trans.mode=batch # Sink Metrics sink.monitor.enable=true sink.monitor.port=9464 # Interact with other rpc -sink.rpc.enable=false -sink.rpc.mock.delay=20 \ No newline at end of file +sink.rpc.enable=true +sink.rpc.mock.delay=20 + +# debezium +debezium.name=testEngine +debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector +debezium.provide.transaction.metadata=true +debezium.offset.storage=org.apache.kafka.connect.storage.FileOffsetBackingStore +debezium.offset.storage.file.filename=/tmp/offsets.dat +debezium.offset.flush.interval.ms=60000 +debezium.schema.history.internal=io.debezium.storage.file.history.FileSchemaHistory +debezium.schema.history.internal.file.filename=/tmp/schemahistory.dat +debezium.database.hostname=localhost +debezium.database.port=5432 +debezium.database.user=pixels +debezium.database.password=pixels_realtime_crud +debezium.database.dbname=pixels_bench_sf1x +debezium.plugin.name=pgoutput +debezium.database.server.id=1 +debezium.schema.include.list=public +debezium.snapshot.mode=never +debezium.key.converter=org.apache.kafka.connect.json.JsonConverter +debezium.value.converter=org.apache.kafka.connect.json.JsonConverter +debezium.topic.prefix=postgresql.oltp_server +debezium.transforms=topicRouting +debezium.transforms.topicRouting.type=org.apache.kafka.connect.transforms.RegexRouter +debezium.transforms.topicRouting.regex=postgresql\\.oltp_server\\.public\\.(.*) +debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf1x.$1 + diff --git a/src/main/resources/pixels-sink.properties b/src/main/resources/pixels-sink.properties index a766bfe..2bad52b 100644 --- a/src/main/resources/pixels-sink.properties +++ b/src/main/resources/pixels-sink.properties @@ -5,12 +5,10 @@ auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer #value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer - # Topic & Database Config topic.prefix=oltp_server consumer.capture_database=pixels_realtime_crud consumer.include_tables= - # Sink Config sink.mode=retina sink.csv.path=./data @@ -21,7 +19,6 @@ sink.batch.size=100 sink.timeout.ms=5000 sink.flush.interval.ms=5000 sink.max.retries=3 - ## Schema Registry sink.registry.url=http://apicurio:8080/apis/registry/v2 # Transaction Config @@ -29,6 +26,8 @@ transaction.topic.suffix=transaction transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer #transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 +## batch or record +sink.trans.mode=batch # Sink Metrics sink.monitor.enable=true sink.monitor.port=9464 diff --git a/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java b/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java new file mode 100644 index 0000000..4e8922b --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java @@ -0,0 +1,98 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink; + + +import io.debezium.config.Configuration; +import io.debezium.embedded.Connect; +import io.debezium.engine.DebeziumEngine; +import io.debezium.engine.RecordChangeEvent; +import io.debezium.engine.format.ChangeEventFormat; +import org.apache.kafka.connect.source.SourceRecord; +import org.junit.jupiter.api.Test; + +import javax.swing.event.ChangeEvent; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * @package: io.pixelsdb.pixels.sink + * @className: DebeziumEngineTest + * @author: AntiO2 + * @date: 2025/9/25 12:16 + */ +public class DebeziumEngineTest +{ + @Test + public void testPostgresCDC() + { + final Properties props = new Properties(); + + props.setProperty("name", "testEngine"); + props.setProperty("connector.class", "io.debezium.connector.postgresql.PostgresConnector"); + props.setProperty("provide.transaction.metadata", "true"); + + props.setProperty("offset.storage", "org.apache.kafka.connect.storage.FileOffsetBackingStore"); + props.setProperty("offset.storage.file.filename", "/tmp/offsets.dat"); + props.setProperty("offset.flush.interval.ms", "60000"); + + props.setProperty("schema.history.internal", "io.debezium.storage.file.history.FileSchemaHistory"); + props.setProperty("schema.history.internal.file.filename", "/tmp/schemahistory.dat"); + + props.setProperty("database.hostname", "localhost"); + props.setProperty("database.port", "5432"); + props.setProperty("database.user", "pixels"); + props.setProperty("database.password", "pixels_realtime_crud"); + props.setProperty("database.dbname", "pixels_bench_sf1x"); + props.setProperty("plugin.name", "pgoutput"); + props.setProperty("database.server.id", "1"); + props.setProperty("schema.include.list", "public"); + props.setProperty("snapshot.mode", "never"); + + props.setProperty("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.setProperty("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.setProperty("topic.prefix", "postgres.cdc"); + + props.setProperty("transforms", "topicRouting"); + props.setProperty("transforms.topicRouting.type", "org.apache.kafka.connect.transforms.RegexRouter"); + props.setProperty("transforms.topicRouting.regex", "postgresql\\.oltp_server\\.public\\.(.*)"); + props.setProperty("transforms.topicRouting.replacement", "postgresql.oltp_server.pixels_bench_sf1x.$1"); + + DebeziumEngine> engine = DebeziumEngine.create(ChangeEventFormat.of(Connect.class)) + .using(props) + .notifying(new MyChangeConsumer()) + .build(); + + ExecutorService executor = Executors.newSingleThreadExecutor(); + executor.execute(engine); + + while (true) { + + } + } + + class MyChangeConsumer implements DebeziumEngine.ChangeConsumer> { + public void handleBatch(List> event, DebeziumEngine.RecordCommitter> committer) throws InterruptedException { + committer.markBatchFinished(); + } + } + +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/TestUtils.java b/src/test/java/io/pixelsdb/pixels/sink/TestUtils.java index b3bd4eb..7713401 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/TestUtils.java +++ b/src/test/java/io/pixelsdb/pixels/sink/TestUtils.java @@ -20,9 +20,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -public class TestUtils { - public static ExecutorService synchronousExecutor() { - return Executors.newSingleThreadExecutor(runnable -> { +public class TestUtils +{ + public static ExecutorService synchronousExecutor() + { + return Executors.newSingleThreadExecutor(runnable -> + { Thread thread = new Thread(runnable); thread.setDaemon(true); return thread; diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java index 4459cc6..74eee8a 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java @@ -21,6 +21,7 @@ import io.pixelsdb.pixels.sink.TestUtils; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -41,47 +42,54 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -class TransactionCoordinatorTest { +class TransactionCoordinatorTest +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionCoordinatorTest.class); private TransactionCoordinator coordinator; private List dispatchedEvents; private ExecutorService testExecutor; private CountDownLatch latch; - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionCoordinatorTest.class); + @BeforeEach - void setUp() throws IOException { + void setUp() throws IOException + { PixelsSinkConfigFactory.initialize(""); testExecutor = TestUtils.synchronousExecutor(); dispatchedEvents = Collections.synchronizedList(new ArrayList<>()); coordinator = new TestableCoordinator(dispatchedEvents); - try { + try + { Field executorField = TransactionCoordinator.class .getDeclaredField("dispatchExecutor"); executorField.setAccessible(true); executorField.set(coordinator, testExecutor); - } catch (Exception e) { + } catch (Exception e) + { throw new RuntimeException("Failed to inject executor", e); } } - private SinkProto.TransactionMetadata buildBeginTx(String txId) { + private SinkProto.TransactionMetadata buildBeginTx(String txId) + { return SinkProto.TransactionMetadata.newBuilder() .setId(txId) .setStatus(SinkProto.TransactionStatus.BEGIN) .build(); } - private SinkProto.TransactionMetadata buildEndTx(String txId) { + private SinkProto.TransactionMetadata buildEndTx(String txId) + { return SinkProto.TransactionMetadata.newBuilder() .setId(txId) .setStatus(SinkProto.TransactionStatus.END) .build(); } - private RowChangeEvent buildEvent(String txId, String table, long collectionOrder, long totalOrder) { + private RowChangeEvent buildEvent(String txId, String table, long collectionOrder, long totalOrder) throws SinkException + { return new RowChangeEvent( - SinkProto.RowRecord.newBuilder().setTransaction( SinkProto.TransactionInfo.newBuilder() .setId(txId) @@ -90,16 +98,17 @@ private RowChangeEvent buildEvent(String txId, String table, long collectionOrde .build() ).setSource( SinkProto.SourceInfo.newBuilder() - .setTable(table) - .setDb("test_db") - .build() + .setTable(table) + .setDb("test_db") + .build() ).setOp(SinkProto.OperationType.INSERT) - .build() + .build(), null ); } @Test - void shouldProcessOrderedEvents() throws Exception { + void shouldProcessOrderedEvents() throws Exception + { coordinator.processTransactionEvent(buildBeginTx("tx1")); coordinator.processRowEvent(buildEvent("tx1", "orders", 1, 1)); @@ -112,7 +121,8 @@ void shouldProcessOrderedEvents() throws Exception { } @Test - void shouldHandleOutOfOrderEvents() { + void shouldHandleOutOfOrderEvents() throws SinkException + { coordinator.processTransactionEvent(buildBeginTx("tx2")); coordinator.processRowEvent(buildEvent("tx2", "users", 3, 3)); coordinator.processRowEvent(buildEvent("tx2", "users", 2, 2)); @@ -124,7 +134,8 @@ void shouldHandleOutOfOrderEvents() { } @Test - void shouldRecoverOrphanedEvents() { + void shouldRecoverOrphanedEvents() throws SinkException + { coordinator.processRowEvent(buildEvent("tx3", "logs", 1, 1)); // orphan event coordinator.processTransactionEvent(buildBeginTx("tx3")); // recover coordinator.processTransactionEvent(buildEndTx("tx3")); @@ -133,11 +144,12 @@ void shouldRecoverOrphanedEvents() { @ParameterizedTest @EnumSource(value = SinkProto.OperationType.class, names = {"INSERT", "UPDATE", "DELETE", "SNAPSHOT"}) - void shouldProcessNonTransactionalEvents(SinkProto.OperationType opType) throws InterruptedException { + void shouldProcessNonTransactionalEvents(SinkProto.OperationType opType) throws InterruptedException, SinkException + { RowChangeEvent event = new RowChangeEvent( SinkProto.RowRecord.newBuilder() .setOp(opType) - .build() + .build(), null ); coordinator.processRowEvent(event); TimeUnit.MILLISECONDS.sleep(10); @@ -146,7 +158,8 @@ void shouldProcessNonTransactionalEvents(SinkProto.OperationType opType) throws } @Test - void shouldHandleTransactionTimeout() throws Exception { + void shouldHandleTransactionTimeout() throws Exception + { TransactionCoordinator fastTimeoutCoordinator = new TransactionCoordinator(); fastTimeoutCoordinator.setTxTimeoutMs(100); @@ -159,17 +172,26 @@ void shouldHandleTransactionTimeout() throws Exception { @ParameterizedTest @ValueSource(ints = {1, 3, 9, 16}) - void shouldHandleConcurrentEvents(int threadCount) throws Exception { + void shouldHandleConcurrentEvents(int threadCount) throws SinkException, IOException, InterruptedException + { PixelsSinkConfigFactory.reset(); PixelsSinkConfigFactory.initialize(""); latch = new CountDownLatch(threadCount); coordinator.processTransactionEvent(buildBeginTx("tx5")); // concurrently send event - for (int i = 0; i < threadCount; i++) { + for (int i = 0; i < threadCount; i++) + { int order = i + 1; - new Thread(() -> { - coordinator.processRowEvent(buildEvent("tx5", "concurrent", order, order)); + new Thread(() -> + { + try + { + coordinator.processRowEvent(buildEvent("tx5", "concurrent", order, order)); + } catch (SinkException e) + { + throw new RuntimeException(e); + } latch.countDown(); }).start(); } @@ -186,17 +208,23 @@ void shouldHandleConcurrentEvents(int threadCount) throws Exception { } - private static class TestableCoordinator extends TransactionCoordinator { - private final List eventLog; + private static class TestableCoordinator extends TransactionCoordinator + { private static final Logger LOGGER = LoggerFactory.getLogger(TestableCoordinator.class); - TestableCoordinator(List eventLog) { + private final List eventLog; + + TestableCoordinator(List eventLog) + { this.eventLog = eventLog; } @Override - protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) { - dispatchExecutor.execute(() -> { - try { + protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) + { + dispatchExecutor.execute(() -> + { + try + { String log = String.format("Dispatching [%s] %s.%s (Order: %s/%s)", event.getOp().name(), event.getDb(), @@ -208,9 +236,12 @@ protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) { LOGGER.info(log); eventLog.add(log); LOGGER.debug("Event log size : {}", eventLog.size()); - } finally { - if (ctx != null) { - if (ctx.pendingEvents.decrementAndGet() == 0 && ctx.completed) { + } finally + { + if (ctx != null) + { + if (ctx.pendingEvents.decrementAndGet() == 0 && ctx.completed) + { ctx.completionFuture.complete(null); } } diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java index 6d710e7..491520b 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java @@ -4,49 +4,86 @@ import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; +import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.List; -public class TransactionServiceTest { +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@Slf4j +public class TransactionServiceTest +{ + private static final Logger logger = LoggerFactory.getLogger(TransactionServiceTest.class); @Test - public void testTransactionService() { + public void testTransactionService() + { int numTransactions = 10; TransService transService = TransService.CreateInstance("localhost", 18889); - try { - List transContexts = transService.beginTransBatch(numTransactions, false); + try + { + List transContexts = transService.beginTransBatch(numTransactions, false); assertEquals(numTransactions, transContexts.size()); TransContext prevTransContext = transContexts.get(0); - for(int i = 1; i < numTransactions; i++) { + for (int i = 1; i < numTransactions; i++) + { TransContext transContext = transContexts.get(i); assertTrue(transContext.getTransId() > prevTransContext.getTransId()); assertTrue(transContext.getTimestamp() > prevTransContext.getTimestamp()); prevTransContext = transContext; } - } catch (TransException e) { + } catch (TransException e) + { throw new RuntimeException(e); } } @Test - public void testBatchRequest() { + public void testBatchRequest() + { int numTransactions = 1000; TransService transService = TransService.CreateInstance("localhost", 18889); - try { - List transContexts = transService.beginTransBatch(numTransactions, false); + try + { + List transContexts = transService.beginTransBatch(numTransactions, false); assertEquals(numTransactions, transContexts.size()); TransContext prevTransContext = transContexts.get(0); - for(int i = 1; i < numTransactions; i++) { + for (int i = 1; i < numTransactions; i++) + { TransContext transContext = transContexts.get(i); assertTrue(transContext.getTransId() > prevTransContext.getTransId()); assertTrue(transContext.getTimestamp() > prevTransContext.getTimestamp()); prevTransContext = transContext; } - } catch (TransException e) { + } catch (TransException e) + { throw new RuntimeException(e); } } + + @Test + public void testAbort() throws TransException + { + TransService transService = TransService.Instance(); + TransContext transContext = transService.beginTrans(true); + + logger.info("ID {}, TS {}", transContext.getTransId(), transContext.getTimestamp()); + TransContext transContext1 = transService.beginTrans(false); + TransContext transContext2 = transService.beginTrans(false); + + logger.info("ID {}, TS {}", transContext1.getTransId(), transContext1.getTimestamp()); + logger.info("ID {}, TS {}", transContext2.getTransId(), transContext2.getTimestamp()); + transService.commitTrans(transContext2.getTransId(), transContext2.getTimestamp()); + + transContext = transService.beginTrans(true); + logger.info("ID {}, TS {}", transContext.getTransId(), transContext.getTimestamp()); + + + } } diff --git a/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java b/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java index 05a2554..a459f79 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java @@ -21,11 +21,11 @@ import io.apicurio.registry.rest.client.RegistryClientFactory; import io.apicurio.registry.serde.SerdeConfig; import io.apicurio.registry.serde.avro.AvroKafkaDeserializer; -import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -40,14 +40,16 @@ import java.util.Collections; import java.util.Properties; -public class AvroConsumerTest { +public class AvroConsumerTest +{ private static final String TOPIC = "oltp_server.pixels_realtime_crud.customer"; private static final String REGISTRY_URL = "http://localhost:8080/apis/registry/v2"; private static final String BOOTSTRAP_SERVERS = "localhost:29092"; private static final String GROUP_ID = "avro-consumer-test-group-1"; - private static KafkaConsumer getRowChangeEventAvroKafkaConsumer() { + private static KafkaConsumer getRowChangeEventAvroKafkaConsumer() + { Properties props = new Properties(); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS); props.put(ConsumerConfig.GROUP_ID_CONFIG, GROUP_ID); @@ -62,15 +64,17 @@ private static KafkaConsumer getRowChangeEventAvroKafkaC return consumer; } - private static void processRecord(RowChangeEvent event) { - RetinaProto.RowValue.Builder builder = RetinaProto.RowValue.newBuilder(); - for (SinkProto.ColumnValue value : event.getRowRecord().getAfter().getValuesList()) { - builder.addValues(value.getValue()); - } - builder.build(); + private static void processRecord(RowChangeEvent event) + { +// RetinaProto.RowValue.Builder builder = RetinaProto.RowValue.newBuilder(); +// for (SinkProto.ColumnValue value : event.getRowRecord().getAfter().getValuesList()) { +// builder.addValues(value.getValue()); +// } +// builder.build(); } - private static KafkaConsumer getStringGenericRecordKafkaConsumer() { + private static KafkaConsumer getStringGenericRecordKafkaConsumer() + { Properties props = new Properties(); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS); props.put(ConsumerConfig.GROUP_ID_CONFIG, GROUP_ID); @@ -85,30 +89,15 @@ private static KafkaConsumer getStringGenericRecordKafkaC return consumer; } - private static RowChangeEvent convertToRowChangeEvent(GenericRecord record, Schema schema) { - return new RowChangeEvent(SinkProto.RowRecord.newBuilder().build()); + private static RowChangeEvent convertToRowChangeEvent(GenericRecord record, Schema schema) throws SinkException + { + return new RowChangeEvent(SinkProto.RowRecord.newBuilder().build(), null); } - @Test - public void avroConsumerTest() { - KafkaConsumer consumer = getStringGenericRecordKafkaConsumer(); - consumer.subscribe(Collections.singletonList(TOPIC)); - - RegistryClient registryClient = RegistryClientFactory.create(REGISTRY_URL); - - try { - while (true) { - ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - for (ConsumerRecord record : records) { - processRecord(record, registryClient); - } - } - } finally { - consumer.close(); - } - } - private static void processRecord(ConsumerRecord record, RegistryClient registryClient) { - try { + private static void processRecord(ConsumerRecord record, RegistryClient registryClient) + { + try + { GenericRecord avroRecord = record.value(); Schema schema = avroRecord.getSchema(); @@ -123,35 +112,68 @@ private static void processRecord(ConsumerRecord record, System.out.println("Offset: " + record.offset()); System.out.println("Event: " + event); - } catch (Exception e) { + } catch (Exception e) + { System.err.println("Error processing message: " + e.getMessage()); e.printStackTrace(); } } - private static String getSchemaIdFromRegistry(RegistryClient client, Schema schema) { + private static String getSchemaIdFromRegistry(RegistryClient client, Schema schema) + { String schemaContent = schema.toString(); - try { + try + { return ""; - } catch (Exception e) { + } catch (Exception e) + { throw new RuntimeException("Schema not found in registry: " + schema.getFullName(), e); } } @Test - public void sinkConsumerTest() throws IOException { + public void avroConsumerTest() + { + KafkaConsumer consumer = getStringGenericRecordKafkaConsumer(); + consumer.subscribe(Collections.singletonList(TOPIC)); + + RegistryClient registryClient = RegistryClientFactory.create(REGISTRY_URL); + + try + { + while (true) + { + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); + for (ConsumerRecord record : records) + { + processRecord(record, registryClient); + } + } + } finally + { + consumer.close(); + } + } + + @Test + public void sinkConsumerTest() throws IOException + { PixelsSinkConfigFactory.initialize("/home/anti/work/pixels-sink/src/main/resources/pixels-sink.local.properties"); KafkaConsumer consumer = getRowChangeEventAvroKafkaConsumer(); consumer.subscribe(Collections.singletonList(TOPIC)); - try { - while (true) { + try + { + while (true) + { ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); - for (ConsumerRecord record : records) { + for (ConsumerRecord record : records) + { processRecord(record.value()); } } - } finally { + } finally + { consumer.close(); } } diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java b/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java new file mode 100644 index 0000000..d9b2e5f --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java @@ -0,0 +1,43 @@ +package io.pixelsdb.pixels.sink.deserializer; + +import io.pixelsdb.pixels.core.TypeDescription; +import io.pixelsdb.pixels.core.vector.BinaryColumnVector; +import io.pixelsdb.pixels.core.vector.VectorizedRowBatch; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +public class RowBatchTest +{ + private final List columnNames = new ArrayList<>(); + private final List columnTypes = new ArrayList<>(); + + @Test + public void integerRowBatchTest() + { + columnNames.add("id"); + columnTypes.add("int"); + + TypeDescription schema = TypeDescription.createSchemaFromStrings(columnNames, columnTypes); + VectorizedRowBatch rowBatch = schema.createRowBatch(3, TypeDescription.Mode.CREATE_INT_VECTOR_FOR_INT); + VectorizedRowBatch newRowBatch = VectorizedRowBatch.deserialize(rowBatch.serialize()); + + } + + @Test + public void varcharRowBatchTest() + { + columnNames.add("name"); + columnTypes.add("varchar(100)"); + + TypeDescription schema = TypeDescription.createSchemaFromStrings(columnNames, columnTypes); + VectorizedRowBatch rowBatch = schema.createRowBatch(3, TypeDescription.Mode.CREATE_INT_VECTOR_FOR_INT); + BinaryColumnVector v = (BinaryColumnVector) rowBatch.cols[0]; + v.add("rr"); + v.add("zz"); + v.add("rr"); + VectorizedRowBatch newRowBatch = VectorizedRowBatch.deserialize(rowBatch.serialize()); + + } +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java b/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java index e85bbb1..32caee2 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java @@ -30,11 +30,13 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; -class RowChangeEventDeserializerTest { +class RowChangeEventDeserializerTest +{ private final Deserializer deserializer = new RowChangeEventJsonDeserializer(); - private String loadSchemaFromFile(String filename) throws IOException, URISyntaxException { + private String loadSchemaFromFile(String filename) throws IOException, URISyntaxException + { ClassLoader classLoader = getClass().getClassLoader(); return new String(Files.readAllBytes(Paths.get( Objects.requireNonNull(classLoader.getResource(filename)).toURI() @@ -58,7 +60,8 @@ private String loadSchemaFromFile(String filename) throws IOException, URISyntax // } @Test - void shouldHandleDeleteOperation() throws Exception { + void shouldHandleDeleteOperation() throws Exception + { String jsonData = loadSchemaFromFile("records/delete.json"); RowChangeEvent event = deserializer.deserialize("test_topic", jsonData.getBytes()); @@ -69,7 +72,8 @@ void shouldHandleDeleteOperation() throws Exception { @Test - void shouldHandleEmptyData() { + void shouldHandleEmptyData() + { RowChangeEvent event = deserializer.deserialize("empty_topic", new byte[0]); assertNull(event); } diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java b/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java index 13d0955..ffc1b6e 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java @@ -17,31 +17,42 @@ package io.pixelsdb.pixels.sink.deserializer; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.TextNode; -import io.pixelsdb.pixels.core.TypeDescription; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; +import io.pixelsdb.pixels.sink.util.DateUtil; +import org.junit.jupiter.api.Test; -import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.Date; -import static org.junit.jupiter.api.Assertions.assertEquals; - -public class RowDataParserTest { +public class RowDataParserTest +{ // BqQ= 17 // JbR7 24710.35 - @ParameterizedTest - @CsvSource({ - // encodedValue, expectedValue, precision, scale - "BqQ=, 17.00, 15, 2", - "JbR7, 24710.35, 15, 2", - }) - void testParseDecimalValid(String encodedValue, String expectedValue, int precision, int scale) { - JsonNode node = new TextNode(encodedValue); - TypeDescription type = TypeDescription.createDecimal(precision, scale); - RowDataParser rowDataParser = new RowDataParser(type); - BigDecimal result = rowDataParser.parseDecimal(node, type); - assertEquals(new BigDecimal(expectedValue), result); - } +// @ParameterizedTest +// @CsvSource({ +// // encodedValue, expectedValue, precision, scale +// "BqQ=, 17.00, 15, 2", +// "JbR7, 24710.35, 15, 2", +// }) +// void testParseDecimalValid(String encodedValue, String expectedValue, int precision, int scale) { +// JsonNode node = new TextNode(encodedValue); +// TypeDescription type = TypeDescription.createDecimal(precision, scale); +// RowDataParser rowDataParser = new RowDataParser(type); +// BigDecimal result = rowDataParser.parseDecimal(node, type); +// assertEquals(new BigDecimal(expectedValue), result); +// } + @Test + void testParseDate() + { + int day = 17059; + Date debeziumDate = DateUtil.fromDebeziumDate(day); + String dayString = DateUtil.convertDateToDayString(debeziumDate); + long ts = 1473927308302000L; + LocalDateTime localDateTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(ts / 1000), ZoneOffset.UTC); + ZonedDateTime zonedDateTime = Instant.ofEpochMilli(ts).atZone(ZoneOffset.UTC); + boolean pause = true; + } } diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java b/src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java index f95b9c1..b6270d0 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java @@ -30,22 +30,26 @@ import java.nio.file.Paths; import java.util.Objects; -public class SchemaDeserializerTest { +public class SchemaDeserializerTest +{ private static final ObjectMapper objectMapper = new ObjectMapper(); @BeforeEach - public void setUp() throws IOException, URISyntaxException { + public void setUp() throws IOException, URISyntaxException + { // String schemaStr = schemaNode.toString(); } - private String loadSchemaFromFile(String filename) throws IOException, URISyntaxException { + private String loadSchemaFromFile(String filename) throws IOException, URISyntaxException + { ClassLoader classLoader = getClass().getClassLoader(); return new String(Files.readAllBytes(Paths.get(Objects.requireNonNull(classLoader.getResource(filename)).toURI()))); } @Test - public void testParseNationStruct() throws IOException, URISyntaxException { + public void testParseNationStruct() throws IOException, URISyntaxException + { String jsonSchema = loadSchemaFromFile("records/nation.json"); JsonNode rootNode = objectMapper.readTree(jsonSchema); JsonNode schemaNode = rootNode.get("schema"); @@ -65,7 +69,8 @@ public void testParseNationStruct() throws IOException, URISyntaxException { } @Test - public void testParseDecimalType() throws IOException { + public void testParseDecimalType() throws IOException + { String jsonSchema = "[{" + "\"type\": \"bytes\"," + "\"name\": \"org.apache.kafka.connect.data.Decimal\"," @@ -86,7 +91,8 @@ public void testParseDecimalType() throws IOException { } @Test - public void testParseDateType() throws IOException { + public void testParseDateType() throws IOException + { String jsonSchema = "[{" + "\"type\": \"int32\"," @@ -101,21 +107,24 @@ public void testParseDateType() throws IOException { // 测试未知类型异常 @Test - public void testParseInvalidType() throws IOException { + public void testParseInvalidType() throws IOException + { String jsonSchema = "[{" + "\"type\": \"unknown_type\"," + "\"field\": \"invalid_field\"" + "}]"; JsonNode rootNode = objectMapper.readTree(jsonSchema); Assertions.assertThrows( - IllegalArgumentException.class, () -> { + IllegalArgumentException.class, () -> + { SchemaDeserializer.parseStruct(rootNode); } ); } @Test - public void testMissingRequiredField() throws IOException { + public void testMissingRequiredField() throws IOException + { String jsonSchema = "{" + "\"type\": \"struct\"," + "\"fields\": [" @@ -124,7 +133,8 @@ public void testMissingRequiredField() throws IOException { + "}"; JsonNode rootNode = objectMapper.readTree(jsonSchema); Assertions.assertThrows( - IllegalArgumentException.class, () -> { + IllegalArgumentException.class, () -> + { SchemaDeserializer.parseFieldType(rootNode); } ); diff --git a/src/test/java/io/pixelsdb/pixels/sink/metadata/TestIndexService.java b/src/test/java/io/pixelsdb/pixels/sink/metadata/TestIndexService.java new file mode 100644 index 0000000..a69a933 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/metadata/TestIndexService.java @@ -0,0 +1,168 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.metadata; + +import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.common.exception.IndexException; +import io.pixelsdb.pixels.common.exception.MetadataException; +import io.pixelsdb.pixels.common.index.IndexService; +import io.pixelsdb.pixels.common.metadata.MetadataService; +import io.pixelsdb.pixels.common.metadata.domain.Layout; +import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; +import io.pixelsdb.pixels.common.metadata.domain.Table; +import io.pixelsdb.pixels.daemon.MetadataProto; +import io.pixelsdb.pixels.index.IndexProto; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; + +/** + * @package: io.pixelsdb.pixels.sink.metadata + * @className: TestIndexService + * @author: AntiO2 + * @date: 2025/8/5 04:34 + */ +public class TestIndexService +{ + + private final MetadataService metadataService = MetadataService.Instance(); + private final IndexService indexService = null; // TODO + + @Test + public void testCreateFreshnessIndex() throws MetadataException + { + String testSchemaName = "pixels_bench_sf1x"; + String testTblName = "freshness"; + String keyColumn = "{\"keyColumnIds\":[15]}"; + Table table = metadataService.getTable(testSchemaName, testTblName); + Layout layout = metadataService.getLatestLayout(testSchemaName, testTblName); + + MetadataProto.SinglePointIndex.Builder singlePointIndexbuilder = MetadataProto.SinglePointIndex.newBuilder(); + singlePointIndexbuilder.setId(0L) + .setKeyColumns(keyColumn) + .setPrimary(true) + .setUnique(true) + .setIndexScheme("rocksdb") + .setTableId(table.getId()) + .setSchemaVersionId(layout.getSchemaVersionId()); + + SinglePointIndex index = new SinglePointIndex(singlePointIndexbuilder.build()); + boolean result = metadataService.createSinglePointIndex(index); + Assertions.assertTrue(result); + boolean pause = true; + } + + @Test + public void testCreateIndex() throws MetadataException + { + String testSchemaName = "pixels_index"; + String testTblName = "ray_index"; + String keyColumn = "{\"keyColumnIds\":[11]}"; + Table table = metadataService.getTable(testSchemaName, testTblName); + long id = table.getId(); + long schemaId = table.getSchemaId(); + Layout layout = metadataService.getLatestLayout(testSchemaName, testTblName); + + MetadataProto.SinglePointIndex.Builder singlePointIndexbuilder = MetadataProto.SinglePointIndex.newBuilder(); + singlePointIndexbuilder.setId(0L) + .setKeyColumns(keyColumn) + .setPrimary(true) + .setUnique(true) + .setIndexScheme("rocksdb") + .setTableId(table.getId()) + .setSchemaVersionId(layout.getSchemaVersionId()); + + SinglePointIndex index = new SinglePointIndex(singlePointIndexbuilder.build()); + boolean result = metadataService.createSinglePointIndex(index); + Assertions.assertTrue(result); + boolean pause = true; + } + + @Test + public void testGetIndex() throws MetadataException + { + String testSchemaName = "pixels_index"; + String testTblName = "ray_index"; + Table table = metadataService.getTable(testSchemaName, testTblName); + long id = table.getId(); + SinglePointIndex index = metadataService.getPrimaryIndex(id); + + Assertions.assertNotNull(index); + boolean pause = true; + } + + @Test + public void testGetRowID() throws MetadataException, IndexException + { + int numRowIds = 10000; + IndexProto.RowIdBatch rowIdBatch = indexService.allocateRowIdBatch(4, numRowIds); + Assertions.assertEquals(rowIdBatch.getLength(), numRowIds); + boolean pause = true; + } + + @Test + public void testPutAndDelete() throws MetadataException, IndexException + { + String table = "customer"; + String db = "pixels_bench_sf1x"; + Table table1 = metadataService.getTable(db, table); + long tableId = table1.getId(); + SinglePointIndex index = metadataService.getPrimaryIndex(tableId); + + String id = "2294222"; + + int len = 1; + int keySize = 0; + keySize += id.length(); + keySize += Long.BYTES + (len + 1) * 2; // table id + index key + + ByteBuffer byteBuffer = ByteBuffer.allocate(keySize); + + byteBuffer.putLong(index.getTableId()).putChar(':'); + byteBuffer.put(id.getBytes()); + byteBuffer.putChar(':'); + + + IndexProto.PrimaryIndexEntry.Builder builder = IndexProto.PrimaryIndexEntry.newBuilder(); + long ts1 = 200000; + long ts2 = 100000; + int rgId = 100; + int rgoffset = 10; + + builder.getIndexKeyBuilder() + .setTimestamp(ts1) + .setKey(ByteString.copyFrom(byteBuffer.rewind())) + .setIndexId(index.getId()) + .setTableId(index.getTableId()); + builder.setRowId(100); + builder.getRowLocationBuilder() + .setRgId(rgId) + .setFileId(0) + .setRgRowOffset(rgoffset); + + boolean b = indexService.putPrimaryIndexEntry(builder.build()); + + builder.getIndexKeyBuilder().setTimestamp(ts2); + + IndexProto.RowLocation rowLocation = indexService.deletePrimaryIndexEntry(builder.getIndexKey()); + + boolean pause = false; + } +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java new file mode 100644 index 0000000..62cf346 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java @@ -0,0 +1,582 @@ +package io.pixelsdb.pixels.sink.sink; + +import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.exception.TransException; +import io.pixelsdb.pixels.common.retina.RetinaService; +import io.pixelsdb.pixels.common.transaction.TransContext; +import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.core.TypeDescription; +import io.pixelsdb.pixels.index.IndexProto; +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.concurrent.TransactionManager; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.util.DateUtil; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class TestRetinaWriter +{ + + static Logger logger = LoggerFactory.getLogger(TestRetinaWriter.class.getName()); + static RetinaService retinaService; + static TableMetadataRegistry metadataRegistry; + static TransService transService; + static int retinaPerformanceTestRowCount; + static int retinaPerformanceTestMaxId; + private final ExecutorService executor = Executors.newFixedThreadPool(16); + + @BeforeAll + public static void setUp() throws IOException + { + PixelsSinkConfigFactory.initialize("/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties"); +// PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties"); + retinaService = RetinaService.Instance(); + metadataRegistry = TableMetadataRegistry.Instance(); + transService = TransService.Instance(); + retinaPerformanceTestRowCount = 5_000_000; + retinaPerformanceTestMaxId = 2_000_000; + } + + @Test + public void insertSingleRecord() throws RetinaException, SinkException, TransException + { + + TransContext ctx = transService.beginTrans(false); + long timeStamp = ctx.getTimestamp(); + String schemaName = "pixels_index"; + String tableName = "ray_index"; + List tableUpdateData = new ArrayList<>(); + RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder = RetinaProto.TableUpdateData.newBuilder(); + tableUpdateDataBuilder.setTableName(tableName); + tableUpdateDataBuilder.setPrimaryIndexId(metadataRegistry.getPrimaryIndexKeyId(schemaName, tableName)); + // + for (int i = 0; i < 10; ++i) + { + byte[][] cols = new byte[3][]; + + cols[0] = Integer.toString(i).getBytes(StandardCharsets.UTF_8); + cols[1] = Long.toString(i * 1000L).getBytes(StandardCharsets.UTF_8); + cols[2] = ("row_" + i).getBytes(StandardCharsets.UTF_8); + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); + afterValueBuilder + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).setName("name").build()); + + + SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); + builder.setOp(SinkProto.OperationType.INSERT) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName) + .build() + ); + RowChangeEvent rowChangeEvent = new RowChangeEvent(builder.build()); + rowChangeEvent.setTimeStamp(timeStamp); + IndexProto.IndexKey indexKey = rowChangeEvent.getAfterKey(); + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); + insertDataBuilder. + addColValues(ByteString.copyFrom((cols[0]))).addColValues(ByteString.copyFrom((cols[1]))) + .addColValues(ByteString.copyFrom((cols[2]))) + .addIndexKeys(indexKey); + tableUpdateDataBuilder.addInsertData(insertDataBuilder.build()); + } + tableUpdateData.add(tableUpdateDataBuilder.build()); + retinaService.updateRecord(schemaName, tableUpdateData); + tableUpdateDataBuilder.setTimestamp(timeStamp); + transService.commitTrans(ctx.getTransId(), timeStamp); + } + + @Test + public void updateSingleRecord() throws RetinaException, SinkException, TransException + { + TransContext ctx = transService.beginTrans(false); + long timeStamp = ctx.getTimestamp(); + String schemaName = "pixels_index"; + String tableName = "ray_index"; + + + List tableUpdateData = new ArrayList<>(); + RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder = RetinaProto.TableUpdateData.newBuilder(); + tableUpdateDataBuilder.setTableName(tableName); + tableUpdateDataBuilder.setPrimaryIndexId(metadataRegistry.getPrimaryIndexKeyId(schemaName, tableName)); + // + for (int i = 0; i < 10; ++i) + { + byte[][] cols = new byte[4][]; + cols[0] = Integer.toString(i).getBytes(StandardCharsets.UTF_8); + cols[1] = Long.toString(i * 1000L).getBytes(StandardCharsets.UTF_8); + cols[2] = ("row_" + i).getBytes(StandardCharsets.UTF_8); + cols[3] = ("updated_row_" + i).getBytes(StandardCharsets.UTF_8); + SinkProto.RowValue.Builder beforeValueBuilder = SinkProto.RowValue.newBuilder(); + beforeValueBuilder + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).setName("name").build()); + + + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); + afterValueBuilder + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[3]))).setName("name").build()); + SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); + builder.setOp(SinkProto.OperationType.UPDATE) + .setBefore(beforeValueBuilder) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName) + .build() + ); + + RowChangeEvent rowChangeEvent = new RowChangeEvent(builder.build()); + rowChangeEvent.setTimeStamp(timeStamp); + RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); + deleteDataBuilder + .addIndexKeys(rowChangeEvent.getBeforeKey()); + tableUpdateDataBuilder.addDeleteData(deleteDataBuilder.build()); + + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); + insertDataBuilder + .addColValues(ByteString.copyFrom((cols[0]))).addColValues(ByteString.copyFrom((cols[1]))) + .addColValues(ByteString.copyFrom((cols[3]))) + .addIndexKeys(rowChangeEvent.getAfterKey()); + tableUpdateDataBuilder.addInsertData(insertDataBuilder.build()); + } + tableUpdateDataBuilder.setTimestamp(timeStamp); + tableUpdateData.add(tableUpdateDataBuilder.build()); + retinaService.updateRecord(schemaName, tableUpdateData); + transService.commitTrans(ctx.getTransId(), timeStamp); + } + + @Test + public void testCheckingAccountInsertPerformance() throws + RetinaException, SinkException, TransException, IOException, ExecutionException, InterruptedException + { + String schemaName = "pixels_bench_sf1x"; + String tableName = "savingaccount"; + + PixelsSinkWriter writer = PixelsSinkWriterFactory.getWriter(); + + TransactionManager manager = TransactionManager.Instance(); + // Step 1: Insert 10,000 records + int totalInserts = retinaPerformanceTestMaxId; + int batchSize = 5; + int batchCount = totalInserts / batchSize; + + int samllBatchCount = 10; + + long start = System.currentTimeMillis(); + + List> futures = new ArrayList<>(); + + for (int b = 0; b < batchCount;) + { + List tableUpdateData = new ArrayList<>(); + for(int sb = 0; sb < samllBatchCount; sb++) + { + ++b; + TransContext ctx = manager.getTransContext(); + long timeStamp = ctx.getTimestamp(); + + RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder = + RetinaProto.TableUpdateData.newBuilder() + .setTableName(tableName) + .setPrimaryIndexId(metadataRegistry.getPrimaryIndexKeyId(schemaName, tableName)); + + for (int i = 0; i < batchSize; i++) + { + int accountID = b * batchSize + i; + int userID = accountID % 1000; + float balance = 1000.0f + accountID; + int isBlocked = 0; + long ts = System.currentTimeMillis(); + + byte[][] cols = new byte[5][]; + cols[0] = ByteBuffer.allocate(Integer.BYTES).putInt(accountID).array(); + cols[1] = ByteBuffer.allocate(Integer.BYTES).putInt(userID).array(); + int intBits = Float.floatToIntBits(balance); + cols[2] = ByteBuffer.allocate(4).putInt(intBits).array(); + cols[3] = ByteBuffer.allocate(Integer.BYTES).putInt(isBlocked).array(); + cols[4] = ByteBuffer.allocate(Long.BYTES).putLong(ts).array(); + // cols[4] = Long.toString(ts).getBytes(StandardCharsets.UTF_8); + // after row + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder() + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[0])).setName("accountid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[1])).setName("userid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[2])).setName("balance").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[3])).setName("isblocked").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[4])).setName("timestamp").build()); + + // RowRecord + SinkProto.RowRecord.Builder rowBuilder = SinkProto.RowRecord.newBuilder() + .setOp(SinkProto.OperationType.INSERT) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName) + .build() + ); + + RowChangeEvent rowChangeEvent = new RowChangeEvent(rowBuilder.build()); + rowChangeEvent.setTimeStamp(timeStamp); + + // InsertData + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder() + .addColValues(ByteString.copyFrom(cols[0])) + .addColValues(ByteString.copyFrom(cols[1])) + .addColValues(ByteString.copyFrom(cols[2])) + .addColValues(ByteString.copyFrom(cols[3])) + .addColValues(ByteString.copyFrom(cols[4])) + .addIndexKeys(rowChangeEvent.getAfterKey()); + + tableUpdateDataBuilder.addInsertData(insertDataBuilder.build()); + } + + tableUpdateData.add(tableUpdateDataBuilder.build()); + + CompletableFuture future = CompletableFuture.runAsync(() -> + { + try + { + transService.commitTrans(ctx.getTransId(), timeStamp); + } catch (TransException e) + { + e.printStackTrace(); + throw new RuntimeException(e); + } + + }, executor); + futures.add(future); + } + Assertions.assertNotNull(writer); + if (!writer.writeBatch(schemaName, tableUpdateData)) + { + logger.error("Error Write Trans"); + System.exit(-1); + } + + } + + for (CompletableFuture future : futures) + { + future.get(); + } + + long end = System.currentTimeMillis(); + double seconds = (end - start) / 1000.0; + double insertsPerSec = totalInserts / seconds; + double transPerSec = batchCount / seconds; + logger.info("Inserted " + totalInserts + " rows in " + seconds + "s, rate=" + insertsPerSec + " inserts/s," + transPerSec + "trans/s"); + writer.close(); + } + + + @Test + public void testCheckingAccountUpdatePerformance() throws + RetinaException, SinkException, TransException, IOException, ExecutionException, InterruptedException + { + String schemaName = "pixels_bench_sf1x"; + String tableName = "checkingaccount"; + + int totalUpdates = retinaPerformanceTestRowCount; // 总更新条数 + int batchSize = 5; // 每个事务包含多少条 update + int batchCount = totalUpdates / batchSize; + + int clientCount = 2; // 可自定义客户端数量 + ExecutorService clientExecutor = Executors.newFixedThreadPool(clientCount); + + List writers = new ArrayList<>(); + for (int c = 0; c < clientCount; c++) + { + writers.add(PixelsSinkWriterFactory.getWriter()); + } + + Random random = new Random(); + TransactionManager manager = TransactionManager.Instance(); + + long start = System.currentTimeMillis(); + List> futures = new ArrayList<>(); + + for (int b = 0; b < batchCount; b++) + { + final int batchIndex = b; + CompletableFuture batchFuture = CompletableFuture.runAsync(() -> + { + try + { + // 轮询选择客户端 + PixelsSinkWriter writer = writers.get(batchIndex % clientCount); + + TransContext ctx = manager.getTransContext(); + long timeStamp = ctx.getTimestamp(); + + List tableUpdateData = new ArrayList<>(); + RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder = + RetinaProto.TableUpdateData.newBuilder() + .setTableName(tableName) + .setPrimaryIndexId(metadataRegistry.getPrimaryIndexKeyId(schemaName, tableName)); + + for (int i = 0; i < batchSize; i++) + { + int accountID = random.nextInt(retinaPerformanceTestMaxId); + int userID = accountID % 1000; + float oldBalance = 1000.0f + accountID; + float newBalance = oldBalance + random.nextInt(1000); + int isBlocked = 0; + long oldTs = System.currentTimeMillis() - 1000; + long newTs = System.currentTimeMillis(); + + // 构建 before/after row + SinkProto.RowValue.Builder beforeValueBuilder = SinkProto.RowValue.newBuilder() + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(accountID))).setName("accountid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(userID))).setName("userid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Float.toString(oldBalance))).setName("balance").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(isBlocked))).setName("isblocked").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(oldTs))).setName("timestamp").build()); + + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder() + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(accountID))).setName("accountid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(userID))).setName("userid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Float.toString(newBalance))).setName("balance").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(isBlocked))).setName("isblocked").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(newTs))).setName("timestamp").build()); + + SinkProto.RowRecord.Builder rowBuilder = SinkProto.RowRecord.newBuilder() + .setOp(SinkProto.OperationType.UPDATE) + .setBefore(beforeValueBuilder) + .setAfter(afterValueBuilder) + .setSource(SinkProto.SourceInfo.newBuilder().setDb(schemaName).setTable(tableName).build()); + + RowChangeEvent rowChangeEvent = new RowChangeEvent(rowBuilder.build()); + rowChangeEvent.setTimeStamp(timeStamp); + + // deleteData + RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder() + .addIndexKeys(rowChangeEvent.getBeforeKey()); + tableUpdateDataBuilder.addDeleteData(deleteDataBuilder.build()); + + // insertData + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder() + .addColValues(ByteString.copyFromUtf8(Integer.toString(accountID))) + .addColValues(ByteString.copyFromUtf8(Integer.toString(userID))) + .addColValues(ByteString.copyFromUtf8(Float.toString(newBalance))) + .addColValues(ByteString.copyFromUtf8(Integer.toString(isBlocked))) + .addColValues(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(newTs))) + .addIndexKeys(rowChangeEvent.getAfterKey()); + tableUpdateDataBuilder.addInsertData(insertDataBuilder.build()); + } + + tableUpdateData.add(tableUpdateDataBuilder.build()); + + long startTime = System.currentTimeMillis(); + if (!writer.writeTrans(schemaName, tableUpdateData, timeStamp)) + { + logger.error("Error Write Trans"); + System.exit(-1); + } + long endTime = System.currentTimeMillis(); + logger.debug("writeTrans batch " + batchIndex + " took " + (endTime - startTime) + " ms"); + + // commit transaction + transService.commitTrans(ctx.getTransId(), timeStamp); + + } catch (Exception e) + { + throw new RuntimeException(e); + } + }, clientExecutor); + + futures.add(batchFuture); + } + + // 等待所有 batch 完成 + for (CompletableFuture f : futures) + { + f.get(); + } + + long end = System.currentTimeMillis(); + double seconds = (end - start) / 1000.0; + double updatesPerSec = totalUpdates / seconds; + double transPerSec = batchCount / seconds; + logger.info("Updated " + totalUpdates + " rows in " + seconds + "s, rate=" + updatesPerSec + " updates/s, " + transPerSec + " trans/s"); + + clientExecutor.shutdown(); + } + + + @Test + public void testInsertTwoTablePerformance() throws + RetinaException, SinkException, TransException, IOException, ExecutionException, InterruptedException + { + String schemaName = "pixels_bench_sf1x"; + String tableName = "checkingaccount"; + String tableName2 = "savingaccount"; + PixelsSinkWriter writer = PixelsSinkWriterFactory.getWriter(); + + TransactionManager manager = TransactionManager.Instance(); + // Step 1: Insert 10,000 records + int totalInserts = retinaPerformanceTestRowCount; + int batchSize = 50; + int batchCount = totalInserts / batchSize; + + + long start = System.currentTimeMillis(); + + List> futures = new ArrayList<>(); + + for (int b = 0; b < batchCount; b++) + { + TransContext ctx = manager.getTransContext(); + long timeStamp = ctx.getTimestamp(); + + List tableUpdateData = new ArrayList<>(); + RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder = + RetinaProto.TableUpdateData.newBuilder() + .setTableName(tableName) + .setPrimaryIndexId(metadataRegistry.getPrimaryIndexKeyId(schemaName, tableName)); + RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder2 = + RetinaProto.TableUpdateData.newBuilder() + .setTableName(tableName2) + .setPrimaryIndexId(metadataRegistry.getPrimaryIndexKeyId(schemaName, tableName2)); + for (int i = 0; i < batchSize; i++) + { + int accountID = b * batchSize + i; + int userID = accountID % 1000; + float balance = 1000.0f + accountID; + int isBlocked = 0; + long ts = System.currentTimeMillis(); + + byte[][] cols = new byte[5][]; + cols[0] = Integer.toString(accountID).getBytes(StandardCharsets.UTF_8); + cols[1] = Integer.toString(userID).getBytes(StandardCharsets.UTF_8); + cols[2] = Float.toString(balance).getBytes(StandardCharsets.UTF_8); + cols[3] = Integer.toString(isBlocked).getBytes(StandardCharsets.UTF_8); + cols[4] = DateUtil.convertDebeziumTimestampToString(ts).getBytes(StandardCharsets.UTF_8); + // cols[4] = Long.toString(ts).getBytes(StandardCharsets.UTF_8); + // after row + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder() + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[0])).setName("accountid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[1])).setName("userid").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[2])).setName("balance").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[3])).setName("isblocked").build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[4])).setName("timestamp").build()); + + // RowRecord + SinkProto.RowRecord.Builder rowBuilder = SinkProto.RowRecord.newBuilder() + .setOp(SinkProto.OperationType.INSERT) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName) + .build() + ); + SinkProto.RowRecord.Builder rowBuilder2 = SinkProto.RowRecord.newBuilder() + .setOp(SinkProto.OperationType.INSERT) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName2) + .build() + ); + RowChangeEvent rowChangeEvent = new RowChangeEvent(rowBuilder.build()); + rowChangeEvent.setTimeStamp(timeStamp); + RowChangeEvent rowChangeEvent2 = new RowChangeEvent(rowBuilder.build()); + rowChangeEvent2.setTimeStamp(timeStamp); + // InsertData + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder() + .addColValues(ByteString.copyFrom(cols[0])) + .addColValues(ByteString.copyFrom(cols[1])) + .addColValues(ByteString.copyFrom(cols[2])) + .addColValues(ByteString.copyFrom(cols[3])) + .addColValues(ByteString.copyFrom(cols[4])) + .addIndexKeys(rowChangeEvent.getAfterKey()); + RetinaProto.InsertData.Builder insertDataBuilder2 = RetinaProto.InsertData.newBuilder() + .addColValues(ByteString.copyFrom(cols[0])) + .addColValues(ByteString.copyFrom(cols[1])) + .addColValues(ByteString.copyFrom(cols[2])) + .addColValues(ByteString.copyFrom(cols[3])) + .addColValues(ByteString.copyFrom(cols[4])) + .addIndexKeys(rowChangeEvent2.getAfterKey()); + tableUpdateDataBuilder.addInsertData(insertDataBuilder.build()); + tableUpdateDataBuilder2.addInsertData(insertDataBuilder2.build()); + } + + tableUpdateData.add(tableUpdateDataBuilder.build()); + tableUpdateData.add(tableUpdateDataBuilder2.build()); + // retinaService.updateRecord(schemaName, tableUpdateData, timeStamp); + long startTime = System.currentTimeMillis(); // 使用 nanoTime 获取更精确的时间 + + + CompletableFuture future = CompletableFuture.runAsync(() -> + { + try + { + // 执行原始的 writeTrans 方法 + writer.writeTrans(schemaName, tableUpdateData, timeStamp); + // 记录结束时间 + long endTime = System.currentTimeMillis(); + + // 计算并输出耗时(单位:毫秒) + long duration = endTime - startTime; + logger.debug("writeTrans took " + duration + " milliseconds"); + transService.commitTrans(ctx.getTransId(), timeStamp); + } catch (TransException e) + { + e.printStackTrace(); + throw new RuntimeException(e); + } + + }, executor); + + futures.add(future); + } + + for (CompletableFuture future : futures) + { + future.get(); + } + + long end = System.currentTimeMillis(); + double seconds = (end - start) / 1000.0; + double insertsPerSec = totalInserts * 2 / seconds; + double transPerSec = batchCount * 2 / seconds; + logger.info("Inserted " + totalInserts + " rows in " + seconds + "s, rate=" + insertsPerSec + " inserts/s," + transPerSec + "trans/s"); + } +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java new file mode 100644 index 0000000..24599fc --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java @@ -0,0 +1,339 @@ +package io.pixelsdb.pixels.sink.sink; + +import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.exception.TransException; +import io.pixelsdb.pixels.common.metadata.MetadataService; +import io.pixelsdb.pixels.common.retina.RetinaService; +import io.pixelsdb.pixels.common.transaction.TransContext; +import io.pixelsdb.pixels.common.transaction.TransService; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.logging.Logger; + +public class TpcHTest +{ + + static Logger logger = Logger.getLogger(TpcHTest.class.getName()); + static RetinaService retinaService; + static MetadataService metadataService; + static TransService transService; + static String schemaName = "pixels_tpch"; + TransContext transContext; + + @BeforeAll + public static void setUp() + { + retinaService = RetinaService.Instance(); + metadataService = MetadataService.Instance(); + transService = TransService.Instance(); + } + + @BeforeEach + public void getTrans() throws TransException + { + transContext = transService.beginTrans(false); + } + + @AfterEach + public void commitTrans() throws TransException + { + transService.commitTrans(transContext.getTransId(), transContext.getTimestamp()); + } + + + @Test + public void insertCustomer() throws RetinaException + { + /* + Column | Type | Extra | Comment + --------------+---------------+-------+--------- + c_custkey | bigint | | + c_name | varchar(25) | | + c_address | varchar(40) | | + c_nationkey | bigint | | + c_phone | char(15) | | + c_acctbal | decimal(15,2) | | + c_mktsegment | char(10) | | + c_comment | varchar(117) | | + (8 rows) + */ + String tableName = "customer"; + for (int i = 0; i < 10; ++i) + { + byte[][] cols = new byte[8][]; + + cols[0] = Long.toString(i).getBytes(StandardCharsets.UTF_8); // c_custkey + cols[1] = ("Customer_" + i).getBytes(StandardCharsets.UTF_8); // c_name + cols[2] = ("Address_" + i).getBytes(StandardCharsets.UTF_8); // c_address + cols[3] = Long.toString(i % 5).getBytes(StandardCharsets.UTF_8); // c_nationkey + cols[4] = String.format("123-456-789%02d", i).getBytes(StandardCharsets.UTF_8); // c_phone (char(15)) + cols[5] = String.format("%.2f", i * 1000.0).getBytes(StandardCharsets.UTF_8); // c_acctbal (decimal) + cols[6] = ("SEGMENT" + (i % 3)).getBytes(StandardCharsets.UTF_8); // c_mktsegment (char(10)) + cols[7] = ("This is customer " + i).getBytes(StandardCharsets.UTF_8); // c_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted customer #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertRegion() throws RetinaException + { + /* + Column | Type | Extra | Comment + ------------+--------------+-------+--------- + r_regionkey | bigint | | + r_name | char(25) | | + r_comment | varchar(152) | | + (3 rows) + */ + String tableName = "region"; + int start = 5; + int end = 10; + for (int i = start; i < end; ++i) + { + byte[][] cols = new byte[3][]; + + cols[0] = Long.toString(i).getBytes(StandardCharsets.UTF_8); // r_regionkey + cols[1] = String.format("Region_%02d", i).getBytes(StandardCharsets.UTF_8); // r_name (char(25)) + cols[2] = ("This is region number " + i).getBytes(StandardCharsets.UTF_8); // r_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted region #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertNation() throws RetinaException + { + /* + Column | Type | Extra | Comment + ------------+--------------+-------+--------- + n_nationkey | bigint | | + n_name | char(25) | | + n_regionkey | bigint | | + n_comment | varchar(152) | | + */ + String tableName = "nation"; + int start = 0; + int end = 10; + + for (int i = start; i < end; ++i) + { + byte[][] cols = new byte[4][]; + + cols[0] = Long.toString(i).getBytes(StandardCharsets.UTF_8); // n_nationkey + cols[1] = String.format("Nation_%02d", i).getBytes(StandardCharsets.UTF_8); // n_name (char(25)) + cols[2] = Long.toString(i % 5).getBytes(StandardCharsets.UTF_8); // n_regionkey + cols[3] = ("This is nation number " + i).getBytes(StandardCharsets.UTF_8); // n_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted nation #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertLineItem() throws RetinaException + { + /* + * Table: lineitem + * Columns: + * l_orderkey | bigint + * l_partkey | bigint + * l_suppkey | bigint + * l_linenumber | integer + * l_quantity | decimal(15,2) + * l_extendedprice | decimal(15,2) + * l_discount | decimal(15,2) + * l_tax | decimal(15,2) + * l_returnflag | char(1) + * l_linestatus | char(1) + * l_shipdate | date + * l_commitdate | date + * l_receiptdate | date + * l_shipinstruct | char(25) + * l_shipmode | char(10) + * l_comment | varchar(44) + */ + String tableName = "lineitem"; + int recordCount = 10; + + for (int i = 0; i < recordCount; ++i) + { + byte[][] cols = new byte[16][]; + + cols[0] = Long.toString(1000 + i).getBytes(StandardCharsets.UTF_8); // l_orderkey + cols[1] = Long.toString(2000 + i).getBytes(StandardCharsets.UTF_8); // l_partkey + cols[2] = Long.toString(3000 + i).getBytes(StandardCharsets.UTF_8); // l_suppkey + cols[3] = Integer.toString(i + 1).getBytes(StandardCharsets.UTF_8); // l_linenumber + cols[4] = String.format("%.2f", 10.0 + i).getBytes(StandardCharsets.UTF_8); // l_quantity + cols[5] = String.format("%.2f", 100.0 + i * 10).getBytes(StandardCharsets.UTF_8); // l_extendedprice + cols[6] = String.format("%.2f", 0.05).getBytes(StandardCharsets.UTF_8); // l_discount + cols[7] = String.format("%.2f", 0.08).getBytes(StandardCharsets.UTF_8); // l_tax + cols[8] = "R".getBytes(StandardCharsets.UTF_8); // l_returnflag + cols[9] = "O".getBytes(StandardCharsets.UTF_8); // l_linestatus + cols[10] = "2025-07-10".getBytes(StandardCharsets.UTF_8); // l_shipdate + cols[11] = "2025-07-12".getBytes(StandardCharsets.UTF_8); // l_commitdate + cols[12] = "2025-07-15".getBytes(StandardCharsets.UTF_8); // l_receiptdate + cols[13] = String.format("DELIVER TO %d", i).getBytes(StandardCharsets.UTF_8); // l_shipinstruct + cols[14] = "AIR".getBytes(StandardCharsets.UTF_8); // l_shipmode + cols[15] = String.format("Order comment %d", i).getBytes(StandardCharsets.UTF_8); // l_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted lineitem #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertOrders() throws RetinaException + { + /* + * Table: orders + * Columns: + * o_orderkey | bigint + * o_custkey | bigint + * o_orderstatus | char(1) + * o_totalprice | decimal(15,2) + * o_orderdate | date + * o_orderpriority | char(15) + * o_clerk | char(15) + * o_shippriority | integer + * o_comment | varchar(79) + */ + String tableName = "orders"; + int recordCount = 10; + + for (int i = 0; i < recordCount; ++i) + { + byte[][] cols = new byte[9][]; + + cols[0] = Long.toString(10000 + i).getBytes(StandardCharsets.UTF_8); // o_orderkey + cols[1] = Long.toString(500 + i).getBytes(StandardCharsets.UTF_8); // o_custkey + cols[2] = "O".getBytes(StandardCharsets.UTF_8); // o_orderstatus + cols[3] = String.format("%.2f", 1234.56 + i * 10).getBytes(StandardCharsets.UTF_8); // o_totalprice + cols[4] = String.format("2025-07-%02d", 10 + i).getBytes(StandardCharsets.UTF_8); // o_orderdate + cols[5] = String.format("PRIORITY-%d", i % 5).getBytes(StandardCharsets.UTF_8); // o_orderpriority + cols[6] = String.format("Clerk#%03d", i).getBytes(StandardCharsets.UTF_8); // o_clerk + cols[7] = Integer.toString(i % 10).getBytes(StandardCharsets.UTF_8); // o_shippriority + cols[8] = String.format("This is order %d", i).getBytes(StandardCharsets.UTF_8); // o_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted order #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertPart() throws RetinaException + { + /* + * Table: part + * Columns: + * p_partkey | bigint + * p_name | varchar(55) + * p_mfgr | char(25) + * p_brand | char(10) + * p_type | varchar(25) + * p_size | integer + * p_container | char(10) + * p_retailprice | decimal(15,2) + * p_comment | varchar(23) + */ + String tableName = "part"; + int recordCount = 10; + + for (int i = 0; i < recordCount; ++i) + { + byte[][] cols = new byte[9][]; + + cols[0] = Long.toString(2000 + i).getBytes(StandardCharsets.UTF_8); // p_partkey + cols[1] = ("PartName_" + i).getBytes(StandardCharsets.UTF_8); // p_name + cols[2] = String.format("MFGR#%02d", i % 5).getBytes(StandardCharsets.UTF_8); // p_mfgr + cols[3] = String.format("Brand#%d", i % 3).getBytes(StandardCharsets.UTF_8); // p_brand + cols[4] = ("TYPE_" + (i % 4)).getBytes(StandardCharsets.UTF_8); // p_type + cols[5] = Integer.toString(5 + i).getBytes(StandardCharsets.UTF_8); // p_size + cols[6] = ("Box" + (i % 6)).getBytes(StandardCharsets.UTF_8); // p_container + cols[7] = String.format("%.2f", 99.99 + i * 2.5).getBytes(StandardCharsets.UTF_8); // p_retailprice + cols[8] = ("Comment_" + i).getBytes(StandardCharsets.UTF_8); // p_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted part #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertPartSupp() throws RetinaException + { + /* + * Table: partsupp + * Columns: + * ps_partkey | bigint + * ps_suppkey | bigint + * ps_availqty | integer + * ps_supplycost | decimal(15,2) + * ps_comment | varchar(199) + */ + String tableName = "partsupp"; + int recordCount = 10; + + for (int i = 0; i < recordCount; ++i) + { + byte[][] cols = new byte[5][]; + + cols[0] = Long.toString(1000 + i).getBytes(StandardCharsets.UTF_8); // ps_partkey + cols[1] = Long.toString(500 + i).getBytes(StandardCharsets.UTF_8); // ps_suppkey + cols[2] = Integer.toString(300 + i * 10).getBytes(StandardCharsets.UTF_8); // ps_availqty + cols[3] = String.format("%.2f", 50.0 + i * 5.5).getBytes(StandardCharsets.UTF_8); // ps_supplycost + cols[4] = ("This is supplier comment #" + i).getBytes(StandardCharsets.UTF_8); // ps_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted partsupp #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + + @Test + public void insertSupplier() throws RetinaException + { + /* + * Table: supplier + * Columns: + * s_suppkey | bigint + * s_name | char(25) + * s_address | varchar(40) + * s_nationkey | bigint + * s_phone | char(15) + * s_acctbal | decimal(15,2) + * s_comment | varchar(101) + */ + String tableName = "supplier"; + int recordCount = 10; + + for (int i = 0; i < recordCount; ++i) + { + byte[][] cols = new byte[7][]; + + cols[0] = Long.toString(2000 + i).getBytes(StandardCharsets.UTF_8); // s_suppkey + cols[1] = String.format("Supplier_%02d", i).getBytes(StandardCharsets.UTF_8); // s_name + cols[2] = ("Address_" + i).getBytes(StandardCharsets.UTF_8); // s_address + cols[3] = Long.toString(i % 5).getBytes(StandardCharsets.UTF_8); // s_nationkey + cols[4] = String.format("987-654-32%03d", i).getBytes(StandardCharsets.UTF_8); // s_phone + cols[5] = String.format("%.2f", 5000.0 + i * 123.45).getBytes(StandardCharsets.UTF_8); // s_acctbal + cols[6] = ("Supplier comment for ID " + i).getBytes(StandardCharsets.UTF_8); // s_comment + +// boolean result = retinaService.insertRecord(schemaName, tableName, cols, transContext.getTimestamp()); +// logger.info("Inserted supplier #" + i + " → result: " + result); +// Assertions.assertTrue(result); + } + } + +} diff --git a/src/test/resources/log4j2.properties b/src/test/resources/log4j2.properties index e91406a..a7675ed 100644 --- a/src/test/resources/log4j2.properties +++ b/src/test/resources/log4j2.properties @@ -1,7 +1,7 @@ status=info name=pixels-sink filter.threshold.type=ThresholdFilter -filter.threshold.level=debug +filter.threshold.level=warn appender.console.type=Console appender.console.name=STDOUT appender.console.layout.type=PatternLayout @@ -12,6 +12,16 @@ appender.rolling.append=true appender.rolling.fileName=${env:PIXELS_HOME}/logs/pixels-sink.log appender.rolling.layout.type=PatternLayout appender.rolling.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n -rootLogger.level=debug +rootLogger.level=info rootLogger.appenderRef.stdout.ref=STDOUT rootLogger.appenderRef.log.ref=log +logger.transaction.name=io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator +logger.transaction.level=info +logger.transaction.appenderRef.log.ref=log +logger.transaction.appenderRef.stdout.ref=STDOUT +logger.transaction.additivity=false +logger.grpc.name=io.grpc.netty.shaded.io.grpc.netty.NettyClientHandler +logger.grpc.level=info +logger.grpc.additivity=false +logger.grpc.appenderRef.log.ref=log +logger.grpc.appenderRef.stdout.ref=STDOUT From f94acc9075075a5a767192d7d46719fa365f5d33 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Mon, 6 Oct 2025 14:31:33 +0000 Subject: [PATCH 02/53] Storage --- .gitignore | 3 + conf/jvm.conf | 2 + pom.xml | 19 ++ .../pixels/sink/PixelsDebeziumConsumer.java | 76 +++++- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 3 + .../concurrent/TransactionCoordinator.java | 10 +- .../sink/concurrent/TransactionManager.java | 5 +- .../pixels/sink/config/PixelsSinkConfig.java | 8 + .../sink/config/PixelsSinkDefaultConfig.java | 1 + .../sink/deserializer/DeserializerUtil.java | 35 --- .../RowChangeEventAvroDeserializer.java | 29 ++- .../RowChangeEventJsonDeserializer.java | 33 +-- .../RowChangeEventStructDeserializer.java | 32 ++- .../sink/deserializer/RowDataParser.java | 75 +++--- .../pixelsdb/pixels/sink/event/ProtoType.java | 56 +++++ .../pixels/sink/event/RowChangeEvent.java | 20 +- .../event/TableEnginePipelineManager.java | 30 ++- .../sink/event/TableEventStorageProvider.java | 101 ++++++++ .../pixels/sink/metadata/TableMetadata.java | 10 + .../sink/metadata/TableMetadataRegistry.java | 51 +++- .../sink/processor/SinkStorageProcessor.java | 220 ++++++++++++++++++ .../sink/processor/TransactionProcessor.java | 2 +- .../pixels/sink/sink/PixelsSinkMode.java | 3 +- .../pixels/sink/sink/ProtoWriter.java | 145 ++++++++++++ .../sink/sink/RotatingWriterManager.java | 92 ++++++++ .../pixels/sink/util/EtcdFileRegistry.java | 182 +++++++++++++++ src/main/resources/log4j2.properties | 2 +- .../resources/pixels-sink.local.properties | 19 +- .../concurrent/TransactionServiceTest.java | 2 +- .../pixels/sink/sink/TestProtoWriter.java | 124 ++++++++++ .../pixels/sink/sink/TestRetinaWriter.java | 10 +- .../pixelsdb/pixels/sink/sink/TpcHTest.java | 2 +- .../sink/util/EtcdFileRegistryTest.java | 53 +++++ 33 files changed, 1292 insertions(+), 163 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java diff --git a/.gitignore b/.gitignore index 0f4acf7..46a9bdd 100644 --- a/.gitignore +++ b/.gitignore @@ -48,3 +48,6 @@ resources/*.xml *.so *.o .vscode + +data/ +tmp/ diff --git a/conf/jvm.conf b/conf/jvm.conf index 011f5d4..8dab3af 100644 --- a/conf/jvm.conf +++ b/conf/jvm.conf @@ -22,3 +22,5 @@ -XX:+ExitOnOutOfMemoryError -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:10086 +--add-opens=java.base/sun.nio.ch=ALL-UNNAMED +--add-opens=java.base/java.nio=ALL-UNNAMED diff --git a/pom.xml b/pom.xml index 5e46b4f..37e269d 100644 --- a/pom.xml +++ b/pom.xml @@ -59,6 +59,19 @@ test + + io.etcd + jetcd-core + true + + + io.netty + netty-all + + + io.grpc + grpc-netty + com.alibaba @@ -216,6 +229,12 @@ 0.16.0 + + + io.pixelsdb + pixels-storage-localfs + + diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java index 955d289..0f969e6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java @@ -24,15 +24,22 @@ import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.deserializer.TransactionStructMessageDeserializer; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.event.TableEnginePipelineManager; import io.pixelsdb.pixels.sink.event.TablePipelineManager; import io.pixelsdb.pixels.sink.event.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.processor.MetricsFacade; import io.pixelsdb.pixels.sink.processor.StoppableProcessor; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; +import io.pixelsdb.pixels.sink.sink.ProtoWriter; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; +import java.io.IOException; import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -54,7 +61,8 @@ public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer> event, if(isTransactionEvent(sourceRecord)) { metricsFacade.recordTransaction(); - // handleTransactionSourceRecord(sourceRecord); + switch (pixelsSinkMode) + { + case RETINA -> + { + handleTransactionSourceRecord(sourceRecord); + } + case PROTO -> + { + SinkProto.TransactionMetadata transactionMetadata = TransactionStructMessageDeserializer.convertToTransactionMetadata(sourceRecord); + protoWriter.writeTrans(transactionMetadata); + } + default -> + { + throw new RuntimeException("Sink Mode " + pixelsSinkMode.toString() + "is not supported"); + } + } } else { metricsFacade.recordRowEvent(); - // handleRowChangeSourceRecord(sourceRecord); + switch (pixelsSinkMode) + { + case RETINA -> + { + handleRowChangeSourceRecord(sourceRecord); + } + case PROTO -> + { + try + { + RowChangeEvent rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(sourceRecord); + protoWriter.write(rowChangeEvent); + } catch (SinkException e) + { + throw new RuntimeException(e); + } + + } + default -> + { + throw new RuntimeException("Sink Mode " + pixelsSinkMode.toString() + "is not supported"); + } + } } } finally @@ -138,5 +198,15 @@ public void stopProcessor() { adapterThread.interrupt(); processor.stopProcessor(); + if(protoWriter != null) + { + try + { + protoWriter.close(); + } catch (IOException e) + { + throw new RuntimeException(e); + } + } } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 3367e59..e5d4942 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -60,6 +60,9 @@ public static void main(String[] args) throws IOException } else if(config.getDataSource().equals("engine")) { mainProcessor = new SinkEngineProcessor(); + } else if(config.getDataSource().equals("storage")) + { + mainProcessor = new SinkStorageProcessor(); } else { throw new IllegalStateException("Unsupported data source type: " + config.getDataSource()); diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java index 6a65182..84862db 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java @@ -174,7 +174,7 @@ private void startTransSync(String sourceTxId) throws SinkException } } ); - LOGGER.info("Begin Tx Sync: {}", sourceTxId); + LOGGER.trace("Begin Tx Sync: {}", sourceTxId); } private void handleOrphanEvents(SinkContext ctx) throws SinkException @@ -220,7 +220,7 @@ private void handleTxEnd(SinkProto.TransactionMetadata txEnd) private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, SinkContext ctx) { - LOGGER.info("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), + LOGGER.trace("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), txEnd.getDataCollectionsList().stream() .map(dc -> dc.getDataCollection() + "=" + ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + @@ -257,7 +257,7 @@ private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, S boolean res = true; if (res) { - LOGGER.info("Committed transaction: {}", txId); + LOGGER.trace("Committed transaction: {}", txId); Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); transactionManager.commitTransAsync(ctx.getPixelsTransCtx()); } else @@ -268,7 +268,7 @@ private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, S { try { - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId()); + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); } catch (TransException e) { throw new RuntimeException(e); @@ -287,7 +287,7 @@ private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, S try { LOGGER.info("Catch Exception, Abort transaction: {}", txId); - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId()); + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); } catch (TransException ex) { LOGGER.error("Failed to abort transaction {}", txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java index 36c0004..a79c1f0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java @@ -107,10 +107,9 @@ public void commitTransAsync(TransContext transContext) try { transService.commitTrans( - transContext.getTransId(), - transContext.getTimestamp() + transContext.getTransId(), false ); - LOGGER.info("Success Commit TXID: {} TS: {}", transContext.getTransId(), transContext.getTimestamp()); + LOGGER.trace("Success Commit TXID: {} TS: {}", transContext.getTransId(), transContext.getTimestamp()); } catch (TransException e) { LOGGER.error("Async commit failed: transId={}", transContext.getTransId()); diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 61ebeff..8fd8094 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -128,6 +128,14 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.datasource", defaultValue = PixelsSinkDefaultConfig.DATA_SOURCE) private String dataSource; + @ConfigKey(value = "sink.proto.dir") + private String sinkProtoDir; + @ConfigKey(value = "sink.proto.data", defaultValue = "data") + private String sinkProtoData; + + @ConfigKey(value = "sink.proto.maxRecords", defaultValue = PixelsSinkDefaultConfig.MAX_RECORDS_PER_FILE) + private int maxRecordsPerFile; + public PixelsSinkConfig(String configFilePath) throws IOException { this.config = ConfigFactory.Instance(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java index 9b649c1..0567daf 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java @@ -60,4 +60,5 @@ public class PixelsSinkDefaultConfig // Mock RPC public static final boolean SINK_RPC_ENABLED = true; public static final int MOCK_RPC_DELAY = 100; + public static final String MAX_RECORDS_PER_FILE = "100000"; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java index 5614ffb..46538d0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java @@ -31,41 +31,6 @@ public class DeserializerUtil { - static RowChangeEvent buildErrorEvent(String topic, byte[] rawData, Exception error) throws SinkException - { - SinkProto.ErrorInfo errorInfo = SinkProto.ErrorInfo.newBuilder() - .setMessage(error.getMessage()) - .setStackTrace(Arrays.toString(error.getStackTrace())) - .setOriginalData(ByteString.copyFrom(rawData)) - .build(); - - SinkProto.RowRecord record = SinkProto.RowRecord.newBuilder() - .setOp(SinkProto.OperationType.UNRECOGNIZED) - .setTsMs(System.currentTimeMillis()) - .build(); - - return new RowChangeEvent(record, null) - { - @Override - public boolean hasError() - { - return true; - } - - @Override - public SinkProto.ErrorInfo getErrorInfo() - { - return errorInfo; - } - - @Override - public String getTopic() - { - return topic; - } - }; - } - static public SinkProto.TransactionStatus getStatusSafely(T record, String field) { String statusString = getStringSafely(record, field); diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java index 1ee7662..bae9000 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java @@ -79,8 +79,8 @@ private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema SinkProto.OperationType op = parseOperationType(avroRecord); SinkProto.RowRecord.Builder recordBuilder = SinkProto.RowRecord.newBuilder() .setOp(op) - .setTsMs(DeserializerUtil.getLongSafely(avroRecord, "ts_ms")); - +// .setTsMs(DeserializerUtil.getLongSafely(avroRecord, "ts_ms")); +; if (avroRecord.get("source") != null) { //TODO: 这里看下怎么处理,如果没有source信息,其实可以通过topic推出schema和table信息。 @@ -136,18 +136,23 @@ private SinkProto.RowValue.Builder parseRowData(Object data, TypeDescription typ private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder builder) { - builder.setVersion(DeserializerUtil.getStringSafely(source, "version")) - .setConnector(DeserializerUtil.getStringSafely(source, "connector")) - .setName(DeserializerUtil.getStringSafely(source, "name")) - .setTsMs(DeserializerUtil.getLongSafely(source, "ts_ms")) - .setSnapshot(DeserializerUtil.getStringSafely(source, "snapshot")) + + builder .setDb(DeserializerUtil.getStringSafely(source, "db")) - .setSequence(DeserializerUtil.getStringSafely(source, "sequence")) - .setSchema(DeserializerUtil.getStringSafely(source, "schema")) + .setSchema(DeserializerUtil.getStringSafely(source, "schema")) .setTable(DeserializerUtil.getStringSafely(source, "table")) - .setTxId(DeserializerUtil.getLongSafely(source, "tx_id")) - .setLsn(DeserializerUtil.getLongSafely(source, "lsn")) - .setXmin(DeserializerUtil.getLongSafely(source, "xmin")); +// .setVersion(DeserializerUtil.getStringSafely(source, "version")) +// .setConnector(DeserializerUtil.getStringSafely(source, "connector")) +// .setName(DeserializerUtil.getStringSafely(source, "name")) +// .setTsMs(DeserializerUtil.getLongSafely(source, "ts_ms")) +// .setSnapshot(DeserializerUtil.getStringSafely(source, "snapshot")) +// +// .setSequence(DeserializerUtil.getStringSafely(source, "sequence")) + +// .setTxId(DeserializerUtil.getLongSafely(source, "tx_id")) +// .setLsn(DeserializerUtil.getLongSafely(source, "lsn")) +// .setXmin(DeserializerUtil.getLongSafely(source, "xmin")) + ; } private void parseTransactionInfo(GenericRecord transaction, diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java index 44802ba..e0a005d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java @@ -83,10 +83,10 @@ private RowChangeEvent buildRowRecord(JsonNode payloadNode, SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); - builder.setOp(parseOperationType(payloadNode)) - .setTsMs(payloadNode.path("ts_ms").asLong()) - .setTsUs(payloadNode.path("ts_us").asLong()) - .setTsNs(payloadNode.path("ts_ns").asLong()); + builder.setOp(parseOperationType(payloadNode)); +// .setTsMs(payloadNode.path("ts_ms").asLong()) +// .setTsUs(payloadNode.path("ts_us").asLong()) +// .setTsNs(payloadNode.path("ts_ns").asLong()); String schemaName; String tableName; @@ -138,20 +138,23 @@ private RowChangeEvent buildRowRecord(JsonNode payloadNode, private SinkProto.SourceInfo.Builder parseSourceInfo(JsonNode sourceNode) { return SinkProto.SourceInfo.newBuilder() - .setVersion(sourceNode.path("version").asText()) - .setConnector(sourceNode.path("connector").asText()) - .setName(sourceNode.path("name").asText()) - .setTsMs(sourceNode.path("ts_ms").asLong()) - .setSnapshot(sourceNode.path("snapshot").asText()) .setDb(sourceNode.path("db").asText()) - .setSequence(sourceNode.path("sequence").asText()) - .setTsUs(sourceNode.path("ts_us").asLong()) - .setTsNs(sourceNode.path("ts_ns").asLong()) .setSchema(sourceNode.path("schema").asText()) .setTable(sourceNode.path("table").asText()) - .setTxId(sourceNode.path("txId").asLong()) - .setLsn(sourceNode.path("lsn").asLong()) - .setXmin(sourceNode.path("xmin").asLong()); +// .setVersion(sourceNode.path("version").asText()) +// .setConnector(sourceNode.path("connector").asText()) +// .setName(sourceNode.path("name").asText()) +// .setTsMs(sourceNode.path("ts_ms").asLong()) +// .setSnapshot(sourceNode.path("snapshot").asText()) + +// .setSequence(sourceNode.path("sequence").asText()) +// .setTsUs(sourceNode.path("ts_us").asLong()) +// .setTsNs(sourceNode.path("ts_ns").asLong()) + +// .setTxId(sourceNode.path("txId").asLong()) +// .setLsn(sourceNode.path("lsn").asLong()) +// .setXmin(sourceNode.path("xmin").asLong()) + ; } private SinkProto.TransactionInfo parseTransactionInfo(JsonNode txNode) diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java index 0aeb2af..6cc0a0b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java @@ -50,6 +50,14 @@ public static RowChangeEvent convertToRowChangeEvent(SourceRecord sourceRecord) return buildRowRecord(value, operationType); } + public static RowChangeEvent convertToRowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException + { + String schemaName = rowRecord.getSource().getDb(); + String tableName = rowRecord.getSource().getTable(); + TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); + return new RowChangeEvent(rowRecord, typeDescription); + } + private static RowChangeEvent buildRowRecord(Struct value, SinkProto.OperationType opType) throws SinkException { @@ -105,20 +113,20 @@ private static RowChangeEvent buildRowRecord(Struct value, private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) { return SinkProto.SourceInfo.newBuilder() - .setVersion(DeserializerUtil.getStringSafely(source, "version")) - .setConnector(DeserializerUtil.getStringSafely(source, "connector")) - .setName(DeserializerUtil.getStringSafely(source, "name")) - .setTsMs(DeserializerUtil.getLongSafely(source, "ts_ms")) - .setSnapshot(DeserializerUtil.getStringSafely(source, "snapshot")) + // .setVersion(DeserializerUtil.getStringSafely(source, "version")) + // .setConnector(DeserializerUtil.getStringSafely(source, "connector")) +// .setName(DeserializerUtil.getStringSafely(source, "name")) +// .setTsMs(DeserializerUtil.getLongSafely(source, "ts_ms")) +// .setSnapshot(DeserializerUtil.getStringSafely(source, "snapshot")) .setDb(DeserializerUtil.getStringSafely(source, "db")) - .setSequence(DeserializerUtil.getStringSafely(source, "sequence")) - .setTsUs(DeserializerUtil.getLongSafely(source, "ts_us")) - .setTsNs(DeserializerUtil.getLongSafely(source, "ts_ns")) +// .setSequence(DeserializerUtil.getStringSafely(source, "sequence")) +// .setTsUs(DeserializerUtil.getLongSafely(source, "ts_us")) +// .setTsNs(DeserializerUtil.getLongSafely(source, "ts_ns")) .setSchema(DeserializerUtil.getStringSafely(source, "schema")) - .setTable(DeserializerUtil.getStringSafely(source, "table")) - .setTxId(DeserializerUtil.getLongSafely(source, "txId")) - .setLsn(DeserializerUtil.getLongSafely(source, "lsn")) - .setXmin(DeserializerUtil.getLongSafely(source, "xmin")); + .setTable(DeserializerUtil.getStringSafely(source, "table")); +// .setTxId(DeserializerUtil.getLongSafely(source, "txId")) +// .setLsn(DeserializerUtil.getLongSafely(source, "lsn")) +// .setXmin(DeserializerUtil.getLongSafely(source, "xmin")); } private static SinkProto.TransactionInfo parseTransactionInfo(T txNode) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java index de79028..db239f9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java +++ b/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java @@ -85,12 +85,11 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel if (valueNode == null || valueNode.isNull()) { return SinkProto.ColumnValue.newBuilder() - .setName(fieldName) + // .setName(fieldName) .setValue(ByteString.EMPTY); } SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); - columnValueBuilder.setName(fieldName); switch (type.getCategory()) { @@ -99,7 +98,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel int value = valueNode.asInt(); byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } case LONG: @@ -107,7 +106,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel long value = valueNode.asLong(); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } case CHAR: @@ -115,8 +114,8 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel String text = valueNode.asText(); byte[] bytes = new byte[] { (byte) text.charAt(0) }; columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder() - .setKind(PixelsProto.Type.Kind.STRING)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder() +// .setKind(PixelsProto.Type.Kind.STRING)); break; } case VARCHAR: @@ -125,24 +124,24 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel { String value = valueNode.asText().trim(); columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } case DECIMAL: { String value = parseDecimal(valueNode, type).toString(); columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder() - .setKind(PixelsProto.Type.Kind.DECIMAL) - .setDimension(type.getPrecision()) - .setScale(type.getScale())); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder() +// .setKind(PixelsProto.Type.Kind.DECIMAL) +// .setDimension(type.getPrecision()) +// .setScale(type.getScale())); break; } case BINARY: { String base64 = valueNode.asText(); // assume already base64 encoded columnValueBuilder.setValue(ByteString.copyFrom(base64, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); break; } case STRUCT: @@ -156,7 +155,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel long longBits = Double.doubleToLongBits(value); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(longBits).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); break; } case FLOAT: @@ -165,7 +164,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel int intBits = Float.floatToIntBits(value); byte[] bytes = ByteBuffer.allocate(4).putInt(intBits).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); break; } case DATE: @@ -173,8 +172,8 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel int isoDate = valueNode.asInt(); byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(isoDate).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder() - .setKind(PixelsProto.Type.Kind.DATE)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder() + // .setKind(PixelsProto.Type.Kind.DATE)); break; } case TIMESTAMP: @@ -182,8 +181,8 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel long timestamp = valueNode.asLong(); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(timestamp).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder() - .setKind(PixelsProto.Type.Kind.DATE)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder() + // .setKind(PixelsProto.Type.Kind.DATE)); break; } default: @@ -198,7 +197,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fieldName, TypeDescription fieldType) { SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); - columnValueBuilder.setName(fieldName); + // columnValueBuilder.setName(fieldName); Object raw = record.get(fieldName); if (raw == null) @@ -213,7 +212,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi { int value = (int) raw; columnValueBuilder.setValue(ByteString.copyFrom(Integer.toString(value), StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } @@ -221,7 +220,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi { long value = (long) raw; columnValueBuilder.setValue(ByteString.copyFrom(Long.toString(value), StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } @@ -229,7 +228,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi { String value = raw.toString(); columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } @@ -238,10 +237,10 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi ByteBuffer buffer = (ByteBuffer) raw; String decimalStr = new String(buffer.array(), StandardCharsets.UTF_8).trim(); columnValueBuilder.setValue(ByteString.copyFrom(decimalStr, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder() - .setKind(PixelsProto.Type.Kind.DECIMAL) - .setDimension(fieldType.getPrecision()) - .setScale(fieldType.getScale())); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder() +// .setKind(PixelsProto.Type.Kind.DECIMAL) +// .setDimension(fieldType.getPrecision()) +// .setScale(fieldType.getScale())); break; } @@ -250,7 +249,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi int epochDay = (int) raw; String isoDate = LocalDate.ofEpochDay(epochDay).toString(); // e.g., "2025-07-03" columnValueBuilder.setValue(ByteString.copyFrom(isoDate, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DATE)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DATE)); break; } @@ -260,7 +259,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi // encode as hex or base64 if needed, otherwise just dump as UTF-8 string if it's meant to be readable String base64 = Base64.getEncoder().encodeToString(buffer.array()); columnValueBuilder.setValue(ByteString.copyFrom(base64, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); break; } default: @@ -276,13 +275,11 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName if (record == null) { return SinkProto.ColumnValue.newBuilder() - .setName(fieldName) + // .setName(fieldName) .setValue(ByteString.EMPTY); } SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); - columnValueBuilder.setName(fieldName); - switch (type) { case INT8: @@ -292,7 +289,7 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName int value = (Integer) record; byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } case INT64: @@ -300,14 +297,14 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName long value = (Long) record; byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } case BYTES: { byte[] bytes = (byte[]) record; columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BYTE)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BYTE)); break; } case BOOLEAN: @@ -315,7 +312,7 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName { String value = (String) record; columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } case STRUCT: @@ -326,10 +323,10 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName case FLOAT64: { double value = (double) record; - long longBits = Double.doubleToLongBits(value); - byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(longBits).array(); + long doubleBits = Double.doubleToLongBits(value); + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(doubleBits).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); break; } case FLOAT32: @@ -338,7 +335,7 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName int intBits = Float.floatToIntBits(value); byte[] bytes = ByteBuffer.allocate(4).putInt(intBits).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); - columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); + // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); break; } default: diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java b/src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java new file mode 100644 index 0000000..f7c4e12 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java @@ -0,0 +1,56 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: ProtoType + * @author: AntiO2 + * @date: 2025/10/5 12:56 + */ +public enum ProtoType +{ + ROW(0), + TRANS(1); + + private final int value; + + ProtoType(int value) + { + this.value = value; + } + + public int toInt() + { + return value; + } + + public static ProtoType fromInt(int value) + { + for (ProtoType type : ProtoType.values()) + { + if (type.value == value) + { + return type; + } + } + throw new IllegalArgumentException("Unknown ProtoType value: " + value); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index f4d22ac..1360042 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -35,6 +35,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class RowChangeEvent { @@ -95,12 +97,8 @@ private void initColumnValueMap() private void initColumnValueMap(SinkProto.RowValue rowValue, Map map) { - rowValue.getValuesList().forEach( - column -> - { - map.put(column.getName(), column); - } - ); + IntStream.range(0, schema.getFieldNames().size()) + .forEach(i -> map.put(schema.getFieldNames().get(i), rowValue.getValuesList().get(i))); } public void initIndexKey() throws SinkException @@ -190,11 +188,6 @@ public boolean hasError() return false; } - public SinkProto.ErrorInfo getErrorInfo() - { - return rowRecord.getError(); - } - public String getDb() { return rowRecord.getSource().getDb(); @@ -230,11 +223,6 @@ public boolean hasAfterData() return isUpdate() || isInsert() || isSnapshot(); } - public Long getTimeStampUs() - { - return rowRecord.getTsUs(); - } - public void startLatencyTimer() { this.latencyTimer = metricsFacade.startProcessLatencyTimer(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java index 549c8bd..7ec2a7b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java @@ -20,9 +20,11 @@ import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.processor.TableProcessor; import org.apache.kafka.connect.source.SourceRecord; +import java.nio.ByteBuffer; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -34,13 +36,14 @@ */ public class TableEnginePipelineManager extends TablePipelineManager { - private final Map pipelines = new ConcurrentHashMap<>(); + private final Map enginePipelines = new ConcurrentHashMap<>(); + private final Map storagePipelines = new ConcurrentHashMap<>(); public void routeRecord(SchemaTableName schemaTableName, SourceRecord record) { - TableEventEngineProvider tableEventEngineProvider = pipelines.computeIfAbsent(schemaTableName, + TableEventEngineProvider tableEventEngineProvider = enginePipelines.computeIfAbsent(schemaTableName, k-> { - TableEventEngineProvider newProvider = createPipeline(k); + TableEventEngineProvider newProvider = createEnginePipeline(k); TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> new TableProcessor(newProvider, schemaTableName)); tableProcessor.run(); @@ -49,9 +52,28 @@ public void routeRecord(SchemaTableName schemaTableName, SourceRecord record) { tableEventEngineProvider.put(record); } - private TableEventEngineProvider createPipeline(SchemaTableName schemaTableName) { + public void routeRecord(SchemaTableName schemaTableName, ByteBuffer record) { + TableEventStorageProvider tableEventStorageProvider = storagePipelines.computeIfAbsent(schemaTableName, + k-> + { + TableEventStorageProvider newProvider = createStoragePipeline(k); + TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> + new TableProcessor(newProvider, schemaTableName)); + tableProcessor.run(); + return newProvider; + }); + tableEventStorageProvider.put(record); + } + + private TableEventEngineProvider createEnginePipeline(SchemaTableName schemaTableName) { TableEventEngineProvider pipeline = new TableEventEngineProvider(schemaTableName); pipeline.start(); return pipeline; } + + private TableEventStorageProvider createStoragePipeline(SchemaTableName schemaTableName) { + TableEventStorageProvider pipeline = new TableEventStorageProvider(schemaTableName); + pipeline.start(); + return pipeline; + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java new file mode 100644 index 0000000..cc2cac5 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java @@ -0,0 +1,101 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +import com.google.protobuf.InvalidProtocolBufferException; +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.apache.kafka.connect.source.SourceRecord; + +import java.nio.ByteBuffer; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.logging.Logger; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEventStorageProvider + * @author: AntiO2 + * @date: 2025/9/26 10:45 + */ +public class TableEventStorageProvider implements TableEventProvider { + private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); + private final SchemaTableName schemaTableName; + + private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(10000); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + private final Thread processorThread; + + public TableEventStorageProvider(SchemaTableName schemaTableName) + { + this.schemaTableName = schemaTableName; + this.processorThread = new Thread(this::processLoop, "TableEventStorageProvider-" + schemaTableName.getTableName()); + } + + @Override + public BlockingQueue getSourceEventQueue() + { + return eventQueue; + } + + public void start() { + processorThread.start(); + } + + public void put(ByteBuffer record) { + try { + rawEventQueue.put(record); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + private void processLoop() { + while (true) { + try { + ByteBuffer data = rawEventQueue.take(); + SinkProto.RowRecord rowRecord = null; + try + { + rowRecord = SinkProto.RowRecord.parseFrom(data); + } catch (InvalidProtocolBufferException e) + { + throw new RuntimeException(e); + } + RowChangeEvent rowChangeEvent = null; + try + { + rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); + } catch (SinkException e) + { + LOGGER.warning(e.getMessage()); + continue; + } + eventQueue.put(rowChangeEvent); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java index bae02a5..24a769d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java @@ -80,4 +80,14 @@ public long getPrimaryIndexKeyId() { return index.getId(); } + + public long getTableId() + { + return table.getId(); + } + + public long getSchemaId() + { + return table.getSchemaId(); + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index d5afd0c..2cd1f2c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -21,6 +21,7 @@ import io.pixelsdb.pixels.common.metadata.MetadataService; import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.common.metadata.domain.Column; +import io.pixelsdb.pixels.common.metadata.domain.Schema; import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.common.metadata.domain.Table; import io.pixelsdb.pixels.core.TypeDescription; @@ -34,11 +35,14 @@ public class TableMetadataRegistry { + private static final Logger logger = LoggerFactory.getLogger(TableMetadataRegistry.class); private static final MetadataService metadataService = MetadataService.Instance(); private static volatile TableMetadataRegistry instance; - private final ConcurrentMap registry = new ConcurrentHashMap<>(); + private final ConcurrentMap registry = new ConcurrentHashMap<>(); + private final ConcurrentMap tableId2SchemaTableName = new ConcurrentHashMap<>(); + private List schemas; private TableMetadataRegistry() { } @@ -71,6 +75,17 @@ public TableMetadata getMetadata(String schema, String table) throws SinkExcepti } + public SchemaTableName getSchemaTableName(long tableId) throws SinkException + { + if (!tableId2SchemaTableName.containsKey(tableId)) + { + logger.info("SchemaTableName doesn't contain {}", tableId); + SchemaTableName metadata = loadSchemaTableName(tableId); + tableId2SchemaTableName.put(tableId, metadata); + } + return tableId2SchemaTableName.get(tableId); + } + public TypeDescription getTypeDescription(String schemaName, String tableName) throws SinkException { return getMetadata(schemaName, tableName).getTypeDescription(); @@ -86,6 +101,13 @@ public long getPrimaryIndexKeyId(String schemaName, String tableName) throws Sin return getMetadata(schemaName, tableName).getPrimaryIndexKeyId(); } + + public long getTableId(String schemaName, String tableName) throws SinkException + { + return getMetadata(schemaName, tableName).getTableId(); + } + + private TableMetadata loadTableMetadata(String schemaName, String tableName) throws SinkException { try @@ -112,4 +134,31 @@ private TableMetadata loadTableMetadata(String schemaName, String tableName) thr throw new SinkException(e); } } + + private SchemaTableName loadSchemaTableName(long tableId) throws SinkException + { + // metadataService + try + { + if(schemas == null) + { + schemas = metadataService.getSchemas(); + } + Table table = metadataService.getTable(tableId); + + long schemaId = table.getSchemaId(); + + Schema schema = schemas.stream() + .filter(s -> s.getId() == schemaId) + .findFirst() + .orElseThrow(() -> new MetadataException("Schema not found for id: " + schemaId)); + + return new SchemaTableName(table.getName(), schema.getName()); + + } catch (MetadataException e) + { + throw new SinkException(e); + } + } + } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java new file mode 100644 index 0000000..f882174 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java @@ -0,0 +1,220 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.processor; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; +import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.ProtoType; +import io.pixelsdb.pixels.sink.event.TableEnginePipelineManager; +import io.pixelsdb.pixels.sink.event.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * @package: io.pixelsdb.pixels.sink.processor + * @className: SinkStorageProcessor + * @author: AntiO2 + * @date: 2025/10/5 11:43 + */ +public class SinkStorageProcessor implements MainProcessor +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SinkStorageProcessor.class); + private final AtomicBoolean running = new AtomicBoolean(false); + + private final String topic; + private final String baseDir; + private final EtcdFileRegistry etcdFileRegistry; + private final List files; + + private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; + private final TableEnginePipelineManager tableEnginePipelineManager = new TableEnginePipelineManager(); + private final TransactionProcessor transactionProcessor = new TransactionProcessor(transactionEventProvider); + private final Thread transactionProcessorThread; + private final Thread transAdapterThread; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + + private final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); + private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + public SinkStorageProcessor() + { + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + this.topic = pixelsSinkConfig.getSinkProtoData(); + this.baseDir = pixelsSinkConfig.getSinkProtoDir(); + this.etcdFileRegistry = new EtcdFileRegistry(topic, baseDir); + this.files = this.etcdFileRegistry.listAllFiles(); + + this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); + this.transAdapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); + + } + + @Override + public void start() + { + this.transactionProcessorThread.start(); + this.transAdapterThread.start(); + ExecutorService transExecutor = Executors.newSingleThreadExecutor(); + ExecutorService rowExecutor = Executors.newSingleThreadExecutor(); + for(String file:files) + { + Storage.Scheme scheme = Storage.Scheme.fromPath(file); + LOGGER.info("Start read from file {}", file); + try(PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) + { + while(true) + { + try { + int keyLen = reader.readInt(ByteOrder.BIG_ENDIAN); + int valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + ByteBuffer keyBuffer = copyToHeap(reader.readFully(keyLen)).order(ByteOrder.BIG_ENDIAN); + ByteBuffer valueBuffer = copyToHeap(reader.readFully(valueLen)).order(ByteOrder.BIG_ENDIAN); + ProtoType protoType = ProtoType.fromInt(keyBuffer.getInt()); + try { + switch (protoType) + { + case ROW -> { + rowExecutor.submit(() -> { + metricsFacade.recordRowEvent(); + handleRowChangeSourceRecord(keyBuffer, valueBuffer); + }); + } + case TRANS -> { + transExecutor.submit(() -> { + metricsFacade.recordTransaction(); + try + { + handleTransactionSourceRecord(valueBuffer); + } catch (InterruptedException e) + { + throw new RuntimeException(e); + } + }); + } + } + } catch (Exception e) { + LOGGER.error("Error processing record", e); + } + } catch (IOException e) { + break; + } + } + } catch (IOException e) + { + throw new RuntimeException(e); + } + + } + } + + private void handleTransactionSourceRecord(ByteBuffer sourceRecord) throws InterruptedException + { + rawTransactionQueue.put(sourceRecord); + } + + + public static ByteBuffer copyToHeap(ByteBuffer directBuffer) { + ByteBuffer duplicate = directBuffer.duplicate(); + ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); + heapBuffer.put(duplicate); + heapBuffer.flip(); + return heapBuffer; + } + private static String readString(ByteBuffer buffer, int len) { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return new String(bytes); + } + + private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBuffer) + { + SchemaTableName schemaTableName = null; + { + // CODE BLOCK VERSION 1 + int schemaLen = keyBuffer.getInt(); + int tableLen = keyBuffer.getInt(); + String schemaName = readString(keyBuffer, schemaLen); + String tableName = readString(keyBuffer, tableLen); + schemaTableName = new SchemaTableName(schemaName, tableName); + } + { + // CODE BLOCK VERSION 2 +// long tableId = keyBuffer.getLong(); +// try +// { +// schemaTableName = tableMetadataRegistry.getSchemaTableName(tableId); +// } catch (SinkException e) +// { +// throw new RuntimeException(e); +// } + } + + tableEnginePipelineManager.routeRecord(schemaTableName, dataBuffer); + } + + private void processTransactionSourceRecord() { + while (true) { + try { + ByteBuffer data = rawTransactionQueue.take(); + SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(data); + if (tx != null) { + transactionEventProvider.getEventQueue().put(tx); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Override + public boolean isRunning() + { + return false; + } + + @Override + public void stopProcessor() + { + transAdapterThread.interrupt(); + transactionProcessor.stopProcessor(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java index 1f16bcb..3134be4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -56,7 +56,7 @@ public void run() SinkProto.TransactionMetadata transaction = transactionEventProvider.getEventQueue().take(); try { - LOGGER.info("Processing transaction event: {}", transaction.getId()); + LOGGER.trace("Processing transaction event: {}", transaction.getId()); transactionCoordinator.processTransactionEvent(transaction); } catch (SinkException e) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java index 10bd82b..b32ac87 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java @@ -21,7 +21,8 @@ public enum PixelsSinkMode { CSV, - RETINA; + RETINA, + PROTO; public static PixelsSinkMode fromValue(String value) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java new file mode 100644 index 0000000..d762334 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java @@ -0,0 +1,145 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.sink; + + +import io.pixelsdb.pixels.common.physical.PhysicalWriter; +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.ProtoType; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; + +/** + * @package: io.pixelsdb.pixels.sink.sink + * @className: ProtoWriter + * @author: AntiO2 + * @date: 2025/10/5 07:10 + */ +public class ProtoWriter implements PixelsSinkWriter +{ + private final Logger LOGGER = LoggerFactory.getLogger(ProtoWriter.class); + private final RotatingWriterManager writerManager; + private final TableMetadataRegistry instance; + + public ProtoWriter() throws IOException + { + PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); + + String dataPath = sinkConfig.getSinkProtoData(); + this.writerManager = new RotatingWriterManager(dataPath); + this.instance = TableMetadataRegistry.Instance(); + } + + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) + { + byte[] transData = transactionMetadata.toByteArray(); + ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); + buffer.putInt(ProtoType.TRANS.toInt()); + return writeData(buffer.array(), transData); + } + + private boolean writeData(byte[] key, byte[] data) + { + ByteBuffer buf = ByteBuffer.allocate(Integer.BYTES + Integer.BYTES + key.length + data.length).order(ByteOrder.BIG_ENDIAN); // rowLen + type + data + + buf.putInt(key.length).putInt(data.length).put(key).put(data); + PhysicalWriter writer; + try { + writer = writerManager.current(); + writer.prepare(buf.remaining()); + writer.append(buf.array()); + } catch (IOException e) + { + LOGGER.error("Error while writing row record.", e); + return false; + } + return true; + } + + @Override + public boolean write(RowChangeEvent rowChangeEvent) + { + return write(rowChangeEvent.getRowRecord()); + } + + public boolean write(SinkProto.RowRecord rowRecord) + { + byte[] rowData = rowRecord.toByteArray(); + String tableName = rowRecord.getSource().getTable(); + String schemaName = rowRecord.getSource().getDb(); + +// long tableId; +// try +// { +// tableId = instance.getTableId(schemaName, tableName); +// } catch (SinkException e) +// { +// LOGGER.error("Error while getting schema table id.", e); +// return false; +// } +// +// ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES + Long.BYTES); +// keyBuffer.putInt(ProtoType.ROW.toInt()) +// .putLong(tableId); + + byte[] schemaNameBytes = schemaName.getBytes(); + byte[] tableNameBytes = tableName.getBytes(); + + ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES * 3 + schemaNameBytes.length + tableNameBytes.length); + keyBuffer.putInt(ProtoType.ROW.toInt()).putInt(schemaNameBytes.length).putInt(tableNameBytes.length); + keyBuffer.put(schemaNameBytes).put(tableNameBytes); + return writeData(keyBuffer.array(), rowData); + } + + @Override + public void flush() + { + + } + + + @Override + public void close() throws IOException + { + this.writerManager.close(); + } + + @Override + public boolean writeTrans(String schemaName, List tableUpdateData, long timestamp) + { + return false; + } + + @Override + public boolean writeBatch(String schemaName, List tableUpdateData) + { + return false; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java b/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java new file mode 100644 index 0000000..66e8c8f --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java @@ -0,0 +1,92 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.sink; + + +import io.pixelsdb.pixels.common.physical.PhysicalWriter; +import io.pixelsdb.pixels.common.physical.PhysicalWriterUtil; +import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; + +import java.io.IOException; + +/** + * @package: io.pixelsdb.pixels.sink.sink + * @className: RotatingWriterManager + * @author: AntiO2 + * @date: 2025/10/5 07:34 + */ +public class RotatingWriterManager +{ + private final String baseDir; + private final String topic; + private final int maxRecordsPerFile; + + private int currentCount = 0; + private PhysicalWriter currentWriter; + private String currentFileName; + + private final Storage.Scheme scheme; + private final EtcdFileRegistry registry; + public RotatingWriterManager(String topic) throws IOException + { + PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); + this.baseDir = sinkConfig.getSinkProtoDir(); + this.topic = topic; + this.maxRecordsPerFile = sinkConfig.getMaxRecordsPerFile(); + this.registry = new EtcdFileRegistry(topic, baseDir); + this.scheme = Storage.Scheme.fromPath(this.baseDir); + rotate(); + } + + private void rotate() throws IOException + { + if (currentWriter != null) + { + currentWriter.close(); + registry.markFileCompleted(registry.getCurrentFileKey()); + } + + currentFileName = registry.createNewFile(); + currentWriter = PhysicalWriterUtil.newPhysicalWriter(scheme, currentFileName); + + currentCount = 0; + } + + public PhysicalWriter current() throws IOException + { + if (currentCount >= maxRecordsPerFile) + { + rotate(); + } + currentCount++; + return currentWriter; + } + + public void close() throws IOException + { + if (currentWriter != null) + { + currentWriter.close(); + registry.markFileCompleted(registry.getCurrentFileKey()); + } + } +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java new file mode 100644 index 0000000..3ae09ea --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java @@ -0,0 +1,182 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.util; + + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.pixelsdb.pixels.common.utils.EtcdUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import io.etcd.jetcd.KeyValue; + +/** + * @package: io.pixelsdb.pixels.sink.util + * @className: EtcdFileRegistry + * @author: AntiO2 + * @date: 2025/10/5 08:24 + */ +public class EtcdFileRegistry { + private static final Logger LOGGER = LoggerFactory.getLogger(EtcdFileRegistry.class); + + private static final String REGISTRY_PREFIX = "/sink/proto/registry/"; + + private final String topic; + private final String baseDir; + private final EtcdUtil etcd = EtcdUtil.Instance(); + private final AtomicInteger nextFileId = new AtomicInteger(0); + private String currentFileKey; + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + public EtcdFileRegistry(String topic, String baseDir) { + this.topic = topic; + this.baseDir = baseDir; + initRegistry(); + } + + private void initRegistry() { + List files = etcd.getKeyValuesByPrefix(filePrefix()); + if (!files.isEmpty()) { + int maxId = files.stream() + .mapToInt(kv -> extractFileId(kv.getKey().toString())) + .max() + .orElse(0); + nextFileId.set(maxId + 1); + LOGGER.info("Initialized registry for topic {} with nextFileId={}", topic, nextFileId.get()); + } else { + LOGGER.info("No existing files found for topic {}, starting fresh", topic); + } + } + + private String topicPrefix() { + return REGISTRY_PREFIX + topic; + } + + private String filePrefix() { + return topicPrefix() + "/files/"; + } + + private int extractFileId(String key) { + try { + String fileName = key.substring(key.lastIndexOf('/') + 1); + String id = fileName.replace(".proto", ""); + return Integer.parseInt(id); + } catch (Exception e) { + return 0; + } + } + + /** + * Create a new file and register it in etcd. + */ + public synchronized String createNewFile() { + String fileName = String.format("%05d.proto", nextFileId.getAndIncrement()); + String fullPath = baseDir + "/" + topic + "/" + fileName; + + Map fileMeta = new HashMap<>(); + fileMeta.put("path", fullPath); + fileMeta.put("created_at", String.valueOf(System.currentTimeMillis())); + fileMeta.put("status", "active"); + currentFileKey = filePrefix() + fileName; + + String jsonValue = null; + try + { + jsonValue = OBJECT_MAPPER.writeValueAsString(fileMeta); + } catch (JsonProcessingException e) + { + throw new RuntimeException(e); + } + + currentFileKey = filePrefix() + fileName; + etcd.putKeyValue(currentFileKey, jsonValue); + etcd.putKeyValue(topicPrefix() + "/current", fileName); + LOGGER.info("Created new file [{}] for topic [{}]", fileName, topic); + return fullPath; + } + + public synchronized String getCurrentFileKey() { + return currentFileKey; + } + + public static String extractPath(String etcdValue) + { + try + { + Map meta = OBJECT_MAPPER.readValue(etcdValue, Map.class); + return (String) meta.get("path"); + } catch (IOException e) + { + LOGGER.error("Failed to parse etcd value: {}", etcdValue, e); + return null; + } + } + /** + * List all files (for readers). + */ + public List listAllFiles() { + List files = etcd.getKeyValuesByPrefix(filePrefix()); + return files.stream() + .map(kv -> { + String value = kv.getValue().toString(); + return extractPath(value); + }) + .sorted() + .collect(Collectors.toList()); + } + + /** + * Mark a file as completed (for writer rotation). + */ + public void markFileCompleted(String fileName) { + KeyValue kv = etcd.getKeyValue(fileName); + if (kv == null) return; + + Map meta = null; + try + { + meta = OBJECT_MAPPER.readValue(kv.getValue().toString(), Map.class); + meta.put("completed_at", String.valueOf(System.currentTimeMillis())); + meta.put("status", "completed"); + String jsonValue = OBJECT_MAPPER.writeValueAsString(meta); + etcd.putKeyValue(fileName, jsonValue); + } catch (JsonProcessingException e) + { + throw new RuntimeException(e); + } + + LOGGER.info("Marked file [{}] as completed", fileName); + } + + public void cleanData() + { + etcd.deleteByPrefix(topicPrefix()); + } +} diff --git a/src/main/resources/log4j2.properties b/src/main/resources/log4j2.properties index 7a254cc..72a2f4d 100644 --- a/src/main/resources/log4j2.properties +++ b/src/main/resources/log4j2.properties @@ -1,7 +1,7 @@ status=info name=pixels-sink filter.threshold.type=ThresholdFilter -filter.threshold.level=warn +filter.threshold.level=info appender.console.type=Console appender.console.name=STDOUT appender.console.layout.type=PatternLayout diff --git a/src/main/resources/pixels-sink.local.properties b/src/main/resources/pixels-sink.local.properties index bfb62b5..9b0cf7d 100644 --- a/src/main/resources/pixels-sink.local.properties +++ b/src/main/resources/pixels-sink.local.properties @@ -1,9 +1,7 @@ -# engine | kafka -sink.datasource=engine - -# embedded engine config - - +# engine | kafka | storage +sink.datasource=storage +# Sink Config: retina | csv | proto | none +sink.mode=proto # Kafka Config bootstrap.servers=localhost:29092 group.id=3107 @@ -15,8 +13,7 @@ value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeseri topic.prefix=postgresql.oltp_server consumer.capture_database=pixels_bench_sf1x consumer.include_tables= -# Sink Config -sink.mode=retina + sink.csv.path=./data sink.csv.enable_header=false ## Retina Config @@ -31,6 +28,12 @@ sink.timeout.ms=5000 sink.flush.interval.ms=100 sink.flush.batch.size=100 sink.max.retries=3 + +## Proto Config +sink.proto.dir=file:///home/pixels/projects/pixels-sink/tmp +sink.proto.data=data +sink.proto.maxRecords=1000000 + ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java index 491520b..6ef868f 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java @@ -79,7 +79,7 @@ public void testAbort() throws TransException logger.info("ID {}, TS {}", transContext1.getTransId(), transContext1.getTimestamp()); logger.info("ID {}, TS {}", transContext2.getTransId(), transContext2.getTimestamp()); - transService.commitTrans(transContext2.getTransId(), transContext2.getTimestamp()); + transService.commitTrans(transContext2.getTransId()); transContext = transService.beginTrans(true); logger.info("ID {}, TS {}", transContext.getTransId(), transContext.getTimestamp()); diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java new file mode 100644 index 0000000..c7b95d1 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java @@ -0,0 +1,124 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.sink; + + +import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.common.retina.RetinaService; +import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.daemon.TransProto; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; +import lombok.SneakyThrows; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +/** + * @package: io.pixelsdb.pixels.sink.sink + * @className: TestProtoWriter + * @author: AntiO2 + * @date: 2025/10/5 09:24 + */ +public class TestProtoWriter +{ + public static String schemaName = "test"; + public static String tableName = "ray"; + + @BeforeAll + public static void setUp() throws IOException + { + PixelsSinkConfigFactory.initialize("/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties"); +// PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties"); + } + @SneakyThrows + @Test + public void testWriteTransInfo() + { + ProtoWriter transWriter = new ProtoWriter(); + int maxTx = 1000; + + for (int i = 0; i < maxTx; i++) + { + transWriter.writeTrans(getTrans(i, SinkProto.TransactionStatus.BEGIN)); + transWriter.writeTrans(getTrans(i, SinkProto.TransactionStatus.END)); + } + transWriter.close(); + } + + @Test + public void testWriteRowInfo() throws IOException + { + ProtoWriter transWriter = new ProtoWriter(); + int maxTx = 10000000; + int rowCnt = 0; + for (int i = 0; i < maxTx; i++) + { + transWriter.writeTrans(getTrans(i, SinkProto.TransactionStatus.BEGIN)); + for(int j = i; j < 3; j++) + { + transWriter.write(getRowRecord(rowCnt++)); + } + transWriter.writeTrans(getTrans(i, SinkProto.TransactionStatus.END)); + } + transWriter.close(); + } + + private static SinkProto.RowRecord getRowRecord(int i) + { + byte[][] cols = new byte[3][]; + + cols[0] = Integer.toString(i).getBytes(StandardCharsets.UTF_8); + cols[1] = Long.toString(i * 1000L).getBytes(StandardCharsets.UTF_8); + cols[2] = ("row_" + i).getBytes(StandardCharsets.UTF_8); + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); + afterValueBuilder + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).setName("name").build()); + + + SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); + builder.setOp(SinkProto.OperationType.INSERT) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName) + .build() + ); + return builder.build(); + } + + private static SinkProto.TransactionMetadata getTrans(int i, SinkProto.TransactionStatus status) + { + SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); + builder.setId(Integer.toString(i)); + builder.setStatus(status); + builder.setTimestamp(System.currentTimeMillis()); + return builder.build(); + } +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java index 62cf346..c60f735 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java @@ -108,7 +108,7 @@ public void insertSingleRecord() throws RetinaException, SinkException, TransExc tableUpdateData.add(tableUpdateDataBuilder.build()); retinaService.updateRecord(schemaName, tableUpdateData); tableUpdateDataBuilder.setTimestamp(timeStamp); - transService.commitTrans(ctx.getTransId(), timeStamp); + transService.commitTrans(ctx.getTransId()); } @Test @@ -178,7 +178,7 @@ public void updateSingleRecord() throws RetinaException, SinkException, TransExc tableUpdateDataBuilder.setTimestamp(timeStamp); tableUpdateData.add(tableUpdateDataBuilder.build()); retinaService.updateRecord(schemaName, tableUpdateData); - transService.commitTrans(ctx.getTransId(), timeStamp); + transService.commitTrans(ctx.getTransId()); } @Test @@ -272,7 +272,7 @@ public void testCheckingAccountInsertPerformance() throws { try { - transService.commitTrans(ctx.getTransId(), timeStamp); + transService.commitTrans(ctx.getTransId()); } catch (TransException e) { e.printStackTrace(); @@ -412,7 +412,7 @@ public void testCheckingAccountUpdatePerformance() throws logger.debug("writeTrans batch " + batchIndex + " took " + (endTime - startTime) + " ms"); // commit transaction - transService.commitTrans(ctx.getTransId(), timeStamp); + transService.commitTrans(ctx.getTransId()); } catch (Exception e) { @@ -556,7 +556,7 @@ public void testInsertTwoTablePerformance() throws // 计算并输出耗时(单位:毫秒) long duration = endTime - startTime; logger.debug("writeTrans took " + duration + " milliseconds"); - transService.commitTrans(ctx.getTransId(), timeStamp); + transService.commitTrans(ctx.getTransId()); } catch (TransException e) { e.printStackTrace(); diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java index 24599fc..b513a34 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java @@ -41,7 +41,7 @@ public void getTrans() throws TransException @AfterEach public void commitTrans() throws TransException { - transService.commitTrans(transContext.getTransId(), transContext.getTimestamp()); + transService.commitTrans(transContext.getTransId()); } diff --git a/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java b/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java new file mode 100644 index 0000000..dee5868 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java @@ -0,0 +1,53 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.util; + + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * @package: io.pixelsdb.pixels.sink.util + * @className: EtcdFileRegistryTest + * @author: AntiO2 + * @date: 2025/10/5 08:54 + */ +public class EtcdFileRegistryTest +{ + private static final Logger LOGGER = LoggerFactory.getLogger(EtcdFileRegistryTest.class); + @Test + public void testCreateFile() + { + EtcdFileRegistry etcdFileRegistry = new EtcdFileRegistry("test", "file:///tmp/test/ray"); + for(int i = 0; i < 10; i++) + { + String newFile = etcdFileRegistry.createNewFile(); + etcdFileRegistry.markFileCompleted(newFile); + } + List files = etcdFileRegistry.listAllFiles(); + for(String file : files) + { + LOGGER.info(file); + } + etcdFileRegistry.cleanData(); + } +} From 88feebfd4914e03add9921e30acf4d93c43e3328 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 7 Oct 2025 08:09:55 +0000 Subject: [PATCH 03/53] Monitor --- .../dashboards/sink-server.json | 41 +++--- pixels-sink | 5 +- .../pixels/sink/PixelsDebeziumConsumer.java | 4 +- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 4 +- .../concurrent/TransactionCoordinator.java | 1 - .../sink/concurrent/TransactionManager.java | 122 +++++++++++++++--- .../pixels/sink/config/PixelsSinkConfig.java | 6 + .../pixels/sink/processor/MetricsFacade.java | 111 +++++++++++++++- .../sink/processor/SinkStorageProcessor.java | 2 - .../pixels/sink/processor/TableProcessor.java | 2 + .../pixels/sink/sink/TableCrossTxWriter.java | 4 +- .../pixels/sink/sink/TableWriter.java | 3 + .../concurrent/TransactionServiceTest.java | 2 +- .../pixels/sink/sink/TestProtoWriter.java | 51 +++++++- .../pixels/sink/sink/TestRetinaWriter.java | 68 +++++----- .../pixelsdb/pixels/sink/sink/TpcHTest.java | 2 +- 16 files changed, 337 insertions(+), 91 deletions(-) diff --git a/develop/images/grafana-provisioning/dashboards/sink-server.json b/develop/images/grafana-provisioning/dashboards/sink-server.json index 69729fc..2d609bb 100644 --- a/develop/images/grafana-provisioning/dashboards/sink-server.json +++ b/develop/images/grafana-provisioning/dashboards/sink-server.json @@ -24,11 +24,25 @@ "links": [], "liveNow": false, "panels": [ + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 26, + "panels": [], + "title": "JVM", + "type": "row" + }, { "datasource": { "type": "prometheus", "uid": "PBFA97CFB590B2093" }, + "description": "", "fieldConfig": { "defaults": { "color": { @@ -86,7 +100,7 @@ "h": 8, "w": 9, "x": 0, - "y": 0 + "y": 1 }, "id": 27, "options": { @@ -115,22 +129,9 @@ "refId": "A" } ], - "title": "Panel Title", + "title": "JVM Heap Memory", "type": "timeseries" }, - { - "collapsed": false, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 8 - }, - "id": 26, - "panels": [], - "title": "JVM", - "type": "row" - }, { "collapsed": false, "gridPos": { @@ -239,7 +240,7 @@ "useBackend": false } ], - "title": "Panel Title", + "title": "Debezium Event Rate", "type": "timeseries" }, { @@ -337,7 +338,7 @@ "useBackend": false } ], - "title": "Panel Title", + "title": "Row Event Rate", "type": "timeseries" }, { @@ -1771,7 +1772,7 @@ "type": "row" } ], - "refresh": "5s", + "refresh": false, "schemaVersion": 38, "style": "light", "tags": [], @@ -1803,8 +1804,8 @@ ] }, "time": { - "from": "now-5m", - "to": "now" + "from": "2025-10-07T04:53:37.436Z", + "to": "2025-10-07T04:55:21.643Z" }, "timepicker": {}, "timezone": "", diff --git a/pixels-sink b/pixels-sink index 754f061..b1c7c10 100755 --- a/pixels-sink +++ b/pixels-sink @@ -6,10 +6,11 @@ SOURCE_PATH=$(readlink -f "$BASH_SOURCE") 2>/dev/null SINK_DIR=$(dirname "$SOURCE_PATH") # Environment variable -export PIXELS_HOME="/home/ubuntu/opt/pixels" +# export PIXELS_HOME="/home/ubuntu/opt/pixels" # Application properties file -PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" +#PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" +PROPERTIES_FILE="/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties" # JVM config file JVM_CONFIG_FILE="${SINK_DIR}/conf/jvm.conf" diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java index 0f969e6..1f29e26 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java @@ -103,7 +103,6 @@ public void handleBatch(List> event, metricsFacade.recordDebeziumEvent(); if(isTransactionEvent(sourceRecord)) { - metricsFacade.recordTransaction(); switch (pixelsSinkMode) { case RETINA -> @@ -112,6 +111,7 @@ public void handleBatch(List> event, } case PROTO -> { + metricsFacade.recordTransaction(); SinkProto.TransactionMetadata transactionMetadata = TransactionStructMessageDeserializer.convertToTransactionMetadata(sourceRecord); protoWriter.writeTrans(transactionMetadata); } @@ -121,7 +121,6 @@ public void handleBatch(List> event, } } } else { - metricsFacade.recordRowEvent(); switch (pixelsSinkMode) { case RETINA -> @@ -134,6 +133,7 @@ public void handleBatch(List> event, { RowChangeEvent rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(sourceRecord); protoWriter.write(rowChangeEvent); + metricsFacade.recordRowEvent(); } catch (SinkException e) { throw new RuntimeException(e); diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index e5d4942..1bcbad0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -17,6 +17,7 @@ package io.pixelsdb.pixels.sink; import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; +import io.pixelsdb.pixels.sink.concurrent.TransactionManager; import io.pixelsdb.pixels.sink.config.CommandLineConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; @@ -42,6 +43,7 @@ public static void main(String[] args) throws IOException { Runtime.getRuntime().addShutdownHook(new Thread(() -> { + TransactionManager.Instance().close(); mainProcessor.stopProcessor(); TransactionCoordinatorFactory.reset(); LOGGER.info("Pixels Sink Server shutdown complete"); @@ -49,7 +51,7 @@ public static void main(String[] args) throws IOException { prometheusHttpServer.close(); } - + MetricsFacade.getInstance().stop(); })); init(args); diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java index 84862db..03ecbf2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java @@ -82,7 +82,6 @@ public void processTransactionEvent(SinkProto.TransactionMetadata txMeta) throws } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) { handleTxEnd(txMeta); - metricsFacade.recordTransaction(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java index a79c1f0..b4bc3f7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java @@ -17,17 +17,18 @@ package io.pixelsdb.pixels.sink.concurrent; +import io.debezium.pipeline.txmetadata.TransactionContext; import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.List; import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.*; /** * This class if for @@ -41,23 +42,33 @@ public class TransactionManager private final TransService transService; private final Queue transContextQueue; private final Object batchLock = new Object(); - private final ExecutorService commitExecutor; + private final ExecutorService batchCommitExecutor; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final BlockingQueue toCommitTransContextQueue; + + private static final int BATCH_SIZE = 100; + private static final int WORKER_COUNT = 16; + private static final int MAX_WAIT_MS = 100; + TransactionManager() { this.transService = TransService.Instance(); this.transContextQueue = new ConcurrentLinkedDeque<>(); - - this.commitExecutor = Executors.newFixedThreadPool( - 4096, - r -> - { + this.toCommitTransContextQueue = new LinkedBlockingQueue<>(); + this.batchCommitExecutor = Executors.newFixedThreadPool( + WORKER_COUNT, + r -> { Thread t = new Thread(r); - t.setName("commit-trans-thread"); + t.setName("commit-trans-batch-thread"); t.setDaemon(true); return t; } ); + for (int i = 0; i < WORKER_COUNT; i++) + { + batchCommitExecutor.submit(this::batchCommitWorker); + } } public static TransactionManager Instance() @@ -102,19 +113,92 @@ public TransContext getTransContext() public void commitTransAsync(TransContext transContext) { - commitExecutor.submit(() -> + toCommitTransContextQueue.add(transContext); + } + + private void batchCommitWorker() + { + List batchTransIds = new ArrayList<>(BATCH_SIZE); + List batchContexts = new ArrayList<>(BATCH_SIZE); + + while (true) { try { - transService.commitTrans( - transContext.getTransId(), false - ); - LOGGER.trace("Success Commit TXID: {} TS: {}", transContext.getTransId(), transContext.getTimestamp()); - } catch (TransException e) + batchContexts.clear(); + batchTransIds.clear(); + + TransContext first = toCommitTransContextQueue.take(); + batchContexts.add(first); + batchTransIds.add(first.getTransId()); + + long startTime = System.nanoTime(); + + while (batchContexts.size() < BATCH_SIZE) + { + long elapsedMs = (System.nanoTime() - startTime) / 1_000_000; + long remainingMs = MAX_WAIT_MS - elapsedMs; + if (remainingMs <= 0) + { + break; + } + + TransContext ctx = toCommitTransContextQueue.poll(remainingMs, TimeUnit.MILLISECONDS); + if (ctx == null) + { + break; + } + batchContexts.add(ctx); + batchTransIds.add(ctx.getTransId()); + } + + transService.commitTransBatch(batchTransIds, false); + metricsFacade.recordTransaction(batchTransIds.size()); + + if (LOGGER.isTraceEnabled()) + { + LOGGER.trace("[{}] Batch committed {} transactions ({} waited ms)", + Thread.currentThread().getName(), + batchTransIds.size(), + (System.nanoTime() - startTime) / 1_000_000); + } + } + catch (InterruptedException ie) + { + LOGGER.warn("Batch commit worker interrupted, exiting..."); + Thread.currentThread().interrupt(); + break; + } + catch (TransException e) + { + LOGGER.error("Batch commit failed: {}", e.getMessage(), e); + } + catch (Exception e) + { + LOGGER.error("Unexpected error in batch commit worker", e); + } + } + } + + public void close() + { + synchronized (batchLock) + { + while(true) { - LOGGER.error("Async commit failed: transId={}", transContext.getTransId()); - e.printStackTrace(); + TransContext ctx = transContextQueue.poll(); + if (ctx == null) + { + break; + } + try + { + transService.rollbackTrans(ctx.getTransId(),false); + } catch (TransException e) + { + throw new RuntimeException(e); + } } - }); + } } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 8fd8094..95270ce 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -71,6 +71,12 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.port", defaultValue = "9464") private short monitorPort; + @ConfigKey(value = "sink.monitor.report.enable", defaultValue = "true") + private boolean monitorReportEnabled; + + @ConfigKey(value = "sink.monitor.report.interval", defaultValue = "5000") + private short monitorReportInterval; + @ConfigKey(value = "sink.rpc.enable", defaultValue = "false") private boolean rpcEnable; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java index 6f142b8..ffcfdf1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java @@ -23,9 +23,15 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.prometheus.client.Counter; import io.prometheus.client.Summary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.atomic.AtomicBoolean; public class MetricsFacade { + private static final Logger LOGGER = LoggerFactory.getLogger(MetricsFacade.class); + private static MetricsFacade instance; private final boolean enabled; private final Counter tableChangeCounter; @@ -42,6 +48,16 @@ public class MetricsFacade private final Summary writerLatency; private final Summary totalLatency; + private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + + private final boolean monitorReportEnabled; + private final int monitorReportInterval; + private final AtomicBoolean running = new AtomicBoolean(false); + private Thread reportThread; + + private long lastRowChangeCount = 0; + private long lastTransactionCount = 0; + private MetricsFacade(boolean enabled) { this.enabled = enabled; @@ -150,11 +166,32 @@ private MetricsFacade(boolean enabled) this.writerLatency = null; this.totalLatency = null; } + + monitorReportEnabled = config.isMonitorReportEnabled(); + monitorReportInterval = config.getMonitorReportInterval(); + + if(monitorReportEnabled) + { + running.set(true); + reportThread = new Thread(this::run, "Metrics Report Thread"); + LOGGER.info("Metrics Report Thread Started"); + reportThread.start(); + } else { + reportThread = null; + } + } + public void stop() + { + running.set(false); + if (reportThread != null) + { + reportThread.interrupt(); + } + LOGGER.info("Monitor report thread stopped."); } private static synchronized void initialize() { - PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); if (instance == null) { instance = new MetricsFacade(config.isMonitorEnabled()); @@ -192,14 +229,19 @@ public void recordRowChange(String table, SinkProto.OperationType operation, int } } - public void recordTransaction() + public void recordTransaction(int i) { if (enabled && transactionCounter != null) { - transactionCounter.inc(); + transactionCounter.inc(i); } } + public void recordTransaction() + { + recordTransaction(1); + } + public Summary.Timer startProcessLatencyTimer() { return enabled ? processingLatency.startTimer() : null; @@ -240,10 +282,71 @@ public void recordTotalLatency(RowChangeEvent event) } public void recordRowEvent() + { + recordRowEvent(1); + } + + public void recordRowEvent(int i) { if (enabled && rowEventCounter != null) { - rowEventCounter.inc(); + rowEventCounter.inc(i); } } + + public int getRecordRowEvent() + { + return (int) rowEventCounter.get(); + } + + public int getTransactionEvent() + { + return (int) transactionCounter.get(); + } + + + public void run() + { + while (running.get()) + { + try + { + logPerformance(); + Thread.sleep(monitorReportInterval); + } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + break; + } + catch (Throwable t) + { + LOGGER.warn("Error while reporting performance.", t); + } + } + } + + + public void logPerformance() + { + long currentRows = (long) rowEventCounter.get(); + long currentTxns = (long) transactionCounter.get(); + + long deltaRows = currentRows - lastRowChangeCount; + long deltaTxns = currentTxns - lastTransactionCount; + + lastRowChangeCount = currentRows; + lastTransactionCount = currentTxns; + + double seconds = monitorReportInterval / 1000.0; + double rowOips = deltaRows / seconds; + double txnOips = deltaTxns / seconds; + + LOGGER.info( + "Performance report: +{} rows (+{}/s), +{} transactions (+{}/s) in {} ms", + deltaRows, String.format("%.2f", rowOips), + deltaTxns, String.format("%.2f", txnOips), + monitorReportInterval + ); + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java index f882174..2ec5995 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java @@ -111,13 +111,11 @@ public void start() { case ROW -> { rowExecutor.submit(() -> { - metricsFacade.recordRowEvent(); handleRowChangeSourceRecord(keyBuffer, valueBuffer); }); } case TRANS -> { transExecutor.submit(() -> { - metricsFacade.recordTransaction(); try { handleTransactionSourceRecord(valueBuffer); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index bc58ceb..20331de 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -44,6 +44,8 @@ public class TableProcessor implements StoppableProcessor, Runnable private final TransactionCoordinator transactionCoordinator; private final TableEventProvider tableEventProvider; private final SchemaTableName schemaTableName; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + public TableProcessor(TableEventProvider tableEventProvider, SchemaTableName schemaTableName) { this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java index 71ad4f9..2f767f1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java @@ -26,6 +26,7 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +50,7 @@ public class TableCrossTxWriter extends TableWriter private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); private final int flushBatchSize; private final ReentrantLock writeLock = new ReentrantLock(); - + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); public TableCrossTxWriter(String t) throws IOException { super(t); @@ -118,6 +119,7 @@ public void flush() resp -> { for(int i = 0; i < txIds.size(); i++) { + metricsFacade.recordRowEvent(tableUpdateCount.get(i)); String writeTxId = txIds.get(i); SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(writeTxId); sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java index 48861b6..755767d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java @@ -25,6 +25,7 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.slf4j.Logger; import java.io.IOException; @@ -56,6 +57,8 @@ public abstract class TableWriter protected ScheduledFuture flushTask = null; protected String fullTableName; protected PixelsSinkConfig config; + private MetricsFacade metricsFacade = MetricsFacade.getInstance(); + protected TableWriter(String tableName) throws IOException { this.config = PixelsSinkConfigFactory.getInstance(); diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java index 6ef868f..d725a03 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java @@ -79,7 +79,7 @@ public void testAbort() throws TransException logger.info("ID {}, TS {}", transContext1.getTransId(), transContext1.getTimestamp()); logger.info("ID {}, TS {}", transContext2.getTransId(), transContext2.getTimestamp()); - transService.commitTrans(transContext2.getTransId()); + transService.commitTrans(transContext2.getTransId(), false); transContext = transService.beginTrans(true); logger.info("ID {}, TS {}", transContext.getTransId(), transContext.getTimestamp()); diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java index c7b95d1..424c1c2 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java @@ -20,6 +20,7 @@ import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.common.physical.*; import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.common.transaction.TransService; import io.pixelsdb.pixels.daemon.TransProto; @@ -27,11 +28,14 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; +import io.pixelsdb.pixels.storage.localfs.PhysicalLocalReader; import lombok.SneakyThrows; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.charset.StandardCharsets; /** @@ -66,6 +70,47 @@ public void testWriteTransInfo() transWriter.close(); } + @Test + public void testWriteFile() throws IOException + { + String path = "/home/pixels/projects/pixels-sink/tmp/write.dat"; + PhysicalWriter writer = PhysicalWriterUtil.newPhysicalWriter(Storage.Scheme.file, path); + + int writeNum = 3; + + ByteBuffer buf = ByteBuffer.allocate(writeNum * Integer.BYTES); + for(int i = 0; i < 3; i++) + { + buf.putInt(i); + } + writer.append(buf); + writer.close(); + } + + + @Test + public void testReadFile() throws IOException + { + String path = "/home/pixels/projects/pixels-sink/tmp/write.dat"; + PhysicalLocalReader reader = (PhysicalLocalReader) PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.file, path); + + int writeNum = 12; + for(int i = 0; i < writeNum; i++) + { + reader.readLong(ByteOrder.BIG_ENDIAN); + } + } + @Test + public void testReadEmptyFile() throws IOException + { + String path = "/home/pixels/projects/pixels-sink/tmp/empty.dat"; + PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.file, path); + + int v = reader.readInt(ByteOrder.BIG_ENDIAN); + + return; + } + @Test public void testWriteRowInfo() throws IOException { @@ -94,11 +139,11 @@ private static SinkProto.RowRecord getRowRecord(int i) SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); afterValueBuilder .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).setName("name").build()); + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).build()); SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java index c60f735..a8aa9b1 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java @@ -79,11 +79,11 @@ public void insertSingleRecord() throws RetinaException, SinkException, TransExc SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); afterValueBuilder .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).setName("name").build()); + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).build()); SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); @@ -108,7 +108,7 @@ public void insertSingleRecord() throws RetinaException, SinkException, TransExc tableUpdateData.add(tableUpdateDataBuilder.build()); retinaService.updateRecord(schemaName, tableUpdateData); tableUpdateDataBuilder.setTimestamp(timeStamp); - transService.commitTrans(ctx.getTransId()); + transService.commitTrans(ctx.getTransId(), false); } @Test @@ -135,21 +135,21 @@ public void updateSingleRecord() throws RetinaException, SinkException, TransExc SinkProto.RowValue.Builder beforeValueBuilder = SinkProto.RowValue.newBuilder(); beforeValueBuilder .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).setName("name").build()); + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).build()); SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); afterValueBuilder .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).setName("id").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).setName("age").build()) + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).build()) .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[3]))).setName("name").build()); + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[3]))).build()); SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); builder.setOp(SinkProto.OperationType.UPDATE) .setBefore(beforeValueBuilder) @@ -178,7 +178,7 @@ public void updateSingleRecord() throws RetinaException, SinkException, TransExc tableUpdateDataBuilder.setTimestamp(timeStamp); tableUpdateData.add(tableUpdateDataBuilder.build()); retinaService.updateRecord(schemaName, tableUpdateData); - transService.commitTrans(ctx.getTransId()); + transService.commitTrans(ctx.getTransId(), false); } @Test @@ -234,11 +234,11 @@ public void testCheckingAccountInsertPerformance() throws // cols[4] = Long.toString(ts).getBytes(StandardCharsets.UTF_8); // after row SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder() - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[0])).setName("accountid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[1])).setName("userid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[2])).setName("balance").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[3])).setName("isblocked").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[4])).setName("timestamp").build()); + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[0])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[1])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[2])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[3])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[4])).build()); // RowRecord SinkProto.RowRecord.Builder rowBuilder = SinkProto.RowRecord.newBuilder() @@ -272,7 +272,7 @@ public void testCheckingAccountInsertPerformance() throws { try { - transService.commitTrans(ctx.getTransId()); + transService.commitTrans(ctx.getTransId(), false); } catch (TransException e) { e.printStackTrace(); @@ -362,18 +362,18 @@ public void testCheckingAccountUpdatePerformance() throws // 构建 before/after row SinkProto.RowValue.Builder beforeValueBuilder = SinkProto.RowValue.newBuilder() - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(accountID))).setName("accountid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(userID))).setName("userid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Float.toString(oldBalance))).setName("balance").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(isBlocked))).setName("isblocked").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(oldTs))).setName("timestamp").build()); + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(accountID))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(userID))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Float.toString(oldBalance))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(isBlocked))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(oldTs))).build()); SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder() - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(accountID))).setName("accountid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(userID))).setName("userid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Float.toString(newBalance))).setName("balance").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(isBlocked))).setName("isblocked").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(newTs))).setName("timestamp").build()); + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(accountID))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(userID))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Float.toString(newBalance))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(Integer.toString(isBlocked))).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFromUtf8(DateUtil.convertDebeziumTimestampToString(newTs))).build()); SinkProto.RowRecord.Builder rowBuilder = SinkProto.RowRecord.newBuilder() .setOp(SinkProto.OperationType.UPDATE) @@ -412,7 +412,7 @@ public void testCheckingAccountUpdatePerformance() throws logger.debug("writeTrans batch " + batchIndex + " took " + (endTime - startTime) + " ms"); // commit transaction - transService.commitTrans(ctx.getTransId()); + transService.commitTrans(ctx.getTransId(), false); } catch (Exception e) { @@ -490,11 +490,11 @@ public void testInsertTwoTablePerformance() throws // cols[4] = Long.toString(ts).getBytes(StandardCharsets.UTF_8); // after row SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder() - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[0])).setName("accountid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[1])).setName("userid").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[2])).setName("balance").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[3])).setName("isblocked").build()) - .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[4])).setName("timestamp").build()); + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[0])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[1])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[2])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[3])).build()) + .addValues(SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom(cols[4])).build()); // RowRecord SinkProto.RowRecord.Builder rowBuilder = SinkProto.RowRecord.newBuilder() @@ -556,7 +556,7 @@ public void testInsertTwoTablePerformance() throws // 计算并输出耗时(单位:毫秒) long duration = endTime - startTime; logger.debug("writeTrans took " + duration + " milliseconds"); - transService.commitTrans(ctx.getTransId()); + transService.commitTrans(ctx.getTransId(), false); } catch (TransException e) { e.printStackTrace(); diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java index b513a34..6cfd446 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java @@ -41,7 +41,7 @@ public void getTrans() throws TransException @AfterEach public void commitTrans() throws TransException { - transService.commitTrans(transContext.getTransId()); + transService.commitTrans(transContext.getTransId(), false); } From a191ea075fe39db294fbbf77341fa770ac9226be Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 8 Oct 2025 06:32:32 +0000 Subject: [PATCH 04/53] 10.7 --- pixels-sink | 4 +- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 2 +- .../sink/concurrent/TransactionManager.java | 15 +- .../pixels/sink/config/PixelsSinkConfig.java | 9 + .../event/TableEnginePipelineManager.java | 4 +- .../sink/event/TableEventStorageProvider.java | 88 +++++-- .../sink/event/TablePipelineManager.java | 1 + .../event/TableStoragePipelineManager.java | 59 +++++ .../AbstractSinkStorageProcessor.java | 39 +++ .../processor/FasterSinkStorageProcessor.java | 241 ++++++++++++++++++ .../pixels/sink/processor/MetricsFacade.java | 64 ++++- .../sink/processor/SinkStorageProcessor.java | 197 +++++++++----- .../pixels/sink/processor/TableProcessor.java | 9 +- .../pixels/sink/sink/PixelsSinkMode.java | 3 +- .../pixels/sink/sink/ProtoWriter.java | 80 +++--- .../pixels/sink/sink/TableCrossTxWriter.java | 26 +- .../pixels/sink/sink/TableWriter.java | 2 + src/main/resources/pixels-sink.aws.properties | 26 +- .../resources/pixels-sink.local.properties | 5 + 19 files changed, 730 insertions(+), 144 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java diff --git a/pixels-sink b/pixels-sink index b1c7c10..cdd3269 100755 --- a/pixels-sink +++ b/pixels-sink @@ -9,8 +9,8 @@ SINK_DIR=$(dirname "$SOURCE_PATH") # export PIXELS_HOME="/home/ubuntu/opt/pixels" # Application properties file -#PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" -PROPERTIES_FILE="/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties" +PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" +#PROPERTIES_FILE="/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties" # JVM config file JVM_CONFIG_FILE="${SINK_DIR}/conf/jvm.conf" diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 1bcbad0..42bc21f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -64,7 +64,7 @@ public static void main(String[] args) throws IOException mainProcessor = new SinkEngineProcessor(); } else if(config.getDataSource().equals("storage")) { - mainProcessor = new SinkStorageProcessor(); + mainProcessor = new FasterSinkStorageProcessor(); } else { throw new IllegalStateException("Unsupported data source type: " + config.getDataSource()); diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java index b4bc3f7..d94f869 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java @@ -21,6 +21,8 @@ import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,13 +48,18 @@ public class TransactionManager private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final BlockingQueue toCommitTransContextQueue; - private static final int BATCH_SIZE = 100; - private static final int WORKER_COUNT = 16; - private static final int MAX_WAIT_MS = 100; + private final int BATCH_SIZE; + private final int WORKER_COUNT; + private final int MAX_WAIT_MS; TransactionManager() { + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + BATCH_SIZE = pixelsSinkConfig.getCommitBatchSize(); + WORKER_COUNT = pixelsSinkConfig.getCommitBatchWorkers(); + MAX_WAIT_MS = pixelsSinkConfig.getCommitBatchDelay(); + this.transService = TransService.Instance(); this.transContextQueue = new ConcurrentLinkedDeque<>(); this.toCommitTransContextQueue = new LinkedBlockingQueue<>(); @@ -69,6 +76,8 @@ public class TransactionManager { batchCommitExecutor.submit(this::batchCommitWorker); } + + } public static TransactionManager Instance() diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 95270ce..d93ba60 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -44,6 +44,15 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) private TransactionMode transactionMode; + @ConfigKey(value = "sink.commit.batch.size", defaultValue = "500") + private int commitBatchSize; + + @ConfigKey(value = "sink.commit.batch.worker", defaultValue = "16") + private int commitBatchWorkers; + + @ConfigKey(value = "sink.commit.batch.delay", defaultValue = "200") + private int commitBatchDelay; + @ConfigKey(value = "sink.remote.port", defaultValue = "9090") private short remotePort; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java index 7ec2a7b..542d87c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java @@ -45,7 +45,7 @@ public void routeRecord(SchemaTableName schemaTableName, SourceRecord record) { { TableEventEngineProvider newProvider = createEnginePipeline(k); TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> - new TableProcessor(newProvider, schemaTableName)); + new TableProcessor(newProvider)); tableProcessor.run(); return newProvider; }); @@ -58,7 +58,7 @@ public void routeRecord(SchemaTableName schemaTableName, ByteBuffer record) { { TableEventStorageProvider newProvider = createStoragePipeline(k); TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> - new TableProcessor(newProvider, schemaTableName)); + new TableProcessor(newProvider)); tableProcessor.run(); return newProvider; }); diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java index cc2cac5..5476ce7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java @@ -24,11 +24,13 @@ import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.apache.kafka.connect.source.SourceRecord; import java.nio.ByteBuffer; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.*; import java.util.logging.Logger; /** @@ -45,10 +47,25 @@ public class TableEventStorageProvider implements TableEventProvider { private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); private final Thread processorThread; + private final MetricsFacade metricsFacade; + + private static final int BATCH_SIZE = 64; + private static final int THREAD_NUM = 4; + private static final long MAX_WAIT_MS = 5; // configurable + private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); + public TableEventStorageProvider(SchemaTableName schemaTableName) { this.schemaTableName = schemaTableName; this.processorThread = new Thread(this::processLoop, "TableEventStorageProvider-" + schemaTableName.getTableName()); + this.metricsFacade = MetricsFacade.getInstance(); + } + + public TableEventStorageProvider(int tableId) + { + this.schemaTableName = null; + this.processorThread = new Thread(this::processLoop, "TableEventStorageProvider-" + tableId); + this.metricsFacade = MetricsFacade.getInstance(); } @Override @@ -70,27 +87,60 @@ public void put(ByteBuffer record) { } private void processLoop() { + List batch = new ArrayList<>(BATCH_SIZE); + while (true) { try { - ByteBuffer data = rawEventQueue.take(); - SinkProto.RowRecord rowRecord = null; - try - { - rowRecord = SinkProto.RowRecord.parseFrom(data); - } catch (InvalidProtocolBufferException e) - { - throw new RuntimeException(e); + batch.clear(); + + // take first element (blocking) + ByteBuffer first = rawEventQueue.take(); + batch.add(first); + long startTime = System.nanoTime(); + + // keep polling until batch full or timeout + while (batch.size() < BATCH_SIZE) { + long elapsedMs = (System.nanoTime() - startTime) / 1_000_000; + long remainingMs = MAX_WAIT_MS - elapsedMs; + if (remainingMs <= 0) { + break; + } + + ByteBuffer next = rawEventQueue.poll(remainingMs, TimeUnit.MILLISECONDS); + if (next == null) { + break; + } + batch.add(next); } - RowChangeEvent rowChangeEvent = null; - try - { - rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); - } catch (SinkException e) - { - LOGGER.warning(e.getMessage()); - continue; + + // parallel decode + List> futures = new ArrayList<>(batch.size()); + for (ByteBuffer data : batch) { + futures.add(decodeExecutor.submit(() -> { + try { + SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(data); + return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } catch (SinkException e) { + LOGGER.warning(e.getMessage()); + return null; + } + })); + } + + // ordered put into queue + for (Future future : futures) { + try { + RowChangeEvent event = future.get(); + if (event != null) { + metricsFacade.recordSerdRowChange(); + eventQueue.put(event); + } + } catch (ExecutionException e) { + LOGGER.warning("Decode failed: " + e.getCause()); + } } - eventQueue.put(rowChangeEvent); } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java index 90ff559..e2aad06 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java @@ -37,4 +37,5 @@ abstract public class TablePipelineManager { protected final ExecutorService executorService = Executors.newCachedThreadPool(); protected final Map activeTableProcessors = new ConcurrentHashMap<>(); + protected final Map id2activeTableProcessors = new ConcurrentHashMap<>(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java new file mode 100644 index 0000000..908d639 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java @@ -0,0 +1,59 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.event; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.processor.TableProcessor; +import org.apache.kafka.connect.source.SourceRecord; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEnginePipelineManager + * @author: AntiO2 + * @date: 2025/9/26 10:44 + */ +public class TableStoragePipelineManager extends TablePipelineManager +{ + private final Map storagePipelines = new ConcurrentHashMap<>(); + + + public void routeRecord(Integer key, ByteBuffer record) { + TableEventStorageProvider tableEventStorageProvider = storagePipelines.computeIfAbsent(key, + k-> + { + TableEventStorageProvider newProvider = createStoragePipeline(k); + TableProcessor tableProcessor = id2activeTableProcessors.computeIfAbsent(k, k2 -> + new TableProcessor(newProvider)); + tableProcessor.run(); + return newProvider; + }); + tableEventStorageProvider.put(record); + } + + private TableEventStorageProvider createStoragePipeline(int key) { + TableEventStorageProvider pipeline = new TableEventStorageProvider(key); + pipeline.start(); + return pipeline; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java new file mode 100644 index 0000000..608d019 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java @@ -0,0 +1,39 @@ +package io.pixelsdb.pixels.sink.processor; + +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.ProtoType; +import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; + +public abstract class AbstractSinkStorageProcessor +{ + protected final AtomicBoolean running = new AtomicBoolean(false); + + protected final String topic; + protected final String baseDir; + protected final EtcdFileRegistry etcdFileRegistry; + protected final List files; + + protected final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); + + protected AbstractSinkStorageProcessor() + { + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + this.topic = pixelsSinkConfig.getSinkProtoData(); + this.baseDir = pixelsSinkConfig.getSinkProtoDir(); + this.etcdFileRegistry = new EtcdFileRegistry(topic, baseDir); + this.files = this.etcdFileRegistry.listAllFiles(); + } + protected void handleTransactionSourceRecord(ByteBuffer sourceRecord) throws InterruptedException + { + rawTransactionQueue.put(sourceRecord); + } + + abstract ProtoType getProtoType(int i); +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java new file mode 100644 index 0000000..39167d7 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java @@ -0,0 +1,241 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.processor; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; +import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.core.utils.Pair; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.*; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +import java.util.concurrent.*; + +/** + * @package: io.pixelsdb.pixels.sink.processor + * @className: SinkStorageProcessor + * @author: AntiO2 + * @date: 2025/10/5 11:43 + */ +public class FasterSinkStorageProcessor extends AbstractSinkStorageProcessor implements MainProcessor +{ + private static final Logger LOGGER = LoggerFactory.getLogger(FasterSinkStorageProcessor.class); + + private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; + private final TableStoragePipelineManager tablePipelineManager = new TableStoragePipelineManager(); + private final TransactionProcessor transactionProcessor = new TransactionProcessor(transactionEventProvider); + private final Thread transactionProcessorThread; + private final Thread transAdapterThread; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final Map>> queueMap = new ConcurrentHashMap<>(); + private final Map consumerThreads = new ConcurrentHashMap<>(); + private final int maxQueueCapacity = 10000; + + + private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + public FasterSinkStorageProcessor() + { + this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); + this.transAdapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); + + } + + @Override + ProtoType getProtoType(int i) + { + if(i == -1) + { + return ProtoType.TRANS; + } + return ProtoType.ROW; + } + + private final CompletableFuture POISON_PILL = new CompletableFuture<>(); + + @Override + public void start() + { + this.transactionProcessorThread.start(); + this.transAdapterThread.start(); + ExecutorService transExecutor = Executors.newSingleThreadExecutor(); + ExecutorService rowExecutor = Executors.newSingleThreadExecutor(); + for(String file:files) + { + Storage.Scheme scheme = Storage.Scheme.fromPath(file); + LOGGER.info("Start read from file {}", file); + try(PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) + { + long offset = 0; + BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); + BlockingQueue> transQueue = new LinkedBlockingQueue<>(); + while(true) + { + try { + int key, valueLen; + reader.seek(offset); + try { + key = reader.readInt(ByteOrder.BIG_ENDIAN); + valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + } catch (IOException e) { + // EOF + break; + } + + ProtoType protoType = getProtoType(key); + offset += Integer.BYTES * 2; + CompletableFuture valueFuture = reader.readAsync(offset, valueLen) + .thenApply(this::copyToHeap) + .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); + // move offset for next record + offset += valueLen; + + + // Get or create queue + BlockingQueue> queue = + queueMap.computeIfAbsent(key, + k -> new LinkedBlockingQueue<>(maxQueueCapacity)); + + // Put future in queue + queue.put(valueFuture); + + // Start consumer thread if not exists + consumerThreads.computeIfAbsent(key, k -> { + Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); + t.setName("consumer-" + key); + t.start(); + return t; + }); + } catch (IOException | InterruptedException e) { + break; + } + } + } catch (IOException e) + { + throw new RuntimeException(e); + } + + } + + // signal all queues to stop + queueMap.values().forEach(q -> { + try { + q.put(POISON_PILL); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + + // wait all consumers to finish + consumerThreads.values().forEach(t -> { + try { + t.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + } + + private ByteBuffer copyToHeap(ByteBuffer directBuffer) { + ByteBuffer duplicate = directBuffer.duplicate(); + ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); + heapBuffer.put(duplicate); + heapBuffer.flip(); + return heapBuffer; + } + private static String readString(ByteBuffer buffer, int len) { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return new String(bytes); + } + + private void consumeQueue(int key, BlockingQueue> queue, ProtoType protoType) { + try { + while (true) { + CompletableFuture value = queue.take(); + if(value == POISON_PILL) + { + break; + } + ByteBuffer valueBuffer = value.get(); + metricsFacade.recordDebeziumEvent(); + switch (protoType) { + case ROW -> handleRowChangeSourceRecord(key, valueBuffer); + case TRANS -> handleTransactionSourceRecord(valueBuffer); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + LOGGER.error("Error in async processing", e); + } + } + static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); + + private void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer) + { + tablePipelineManager.routeRecord(key, dataBuffer); + } + + private void processTransactionSourceRecord() { + while (true) { + ByteBuffer data = null; + try + { + data = rawTransactionQueue.take(); + } catch (InterruptedException e) + { + throw new RuntimeException(e); + } + try { + SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(data); + metricsFacade.recordSerdTxChange(); + if (tx != null) { + transactionEventProvider.getEventQueue().put(tx); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Override + public boolean isRunning() + { + return false; + } + + @Override + public void stopProcessor() + { + transAdapterThread.interrupt(); + transactionProcessor.stopProcessor(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java index ffcfdf1..10f45a8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java @@ -37,6 +37,8 @@ public class MetricsFacade private final Counter tableChangeCounter; private final Counter rowChangeCounter; private final Counter transactionCounter; + private final Counter serdRowRecordCounter; + private final Counter serdTxRecordCounter; private final Summary processingLatency; private final Counter rawDataThroughputCounter; private final Counter debeziumEventCounter; @@ -57,6 +59,9 @@ public class MetricsFacade private long lastRowChangeCount = 0; private long lastTransactionCount = 0; + private long lastDebeziumCount = 0; + private long lastSerdRowRecordCount = 0; + private long lastSerdTxRecordCount = 0; private MetricsFacade(boolean enabled) { @@ -73,6 +78,15 @@ private MetricsFacade(boolean enabled) .help("Debezium Row Event Total") .register(); + this.serdRowRecordCounter = Counter.build() + .name("serd_row_record") + .help("Serialized Row Record Total") + .register(); + + this.serdTxRecordCounter = Counter.build() + .name("serd_tx_record") + .help("Serialized Transaction Record Total") + .register(); this.tableChangeCounter = Counter.build() .name("sink_table_changes_total") @@ -156,6 +170,8 @@ private MetricsFacade(boolean enabled) this.debeziumEventCounter = null; this.rowEventCounter = null; this.rowChangeCounter = null; + this.serdRowRecordCounter = null; + this.serdTxRecordCounter = null; this.transactionCounter = null; this.processingLatency = null; this.tableChangeCounter = null; @@ -229,6 +245,34 @@ public void recordRowChange(String table, SinkProto.OperationType operation, int } } + public void recordSerdRowChange() + { + recordSerdRowChange(1); + } + + public void recordSerdRowChange(int i) + { + if (enabled && serdRowRecordCounter != null) + { + serdRowRecordCounter.inc(i); + } + } + + + public void recordSerdTxChange() + { + recordSerdTxChange(1); + } + + public void recordSerdTxChange(int i) + { + if (enabled && serdTxRecordCounter != null) + { + serdTxRecordCounter.inc(i); + } + } + + public void recordTransaction(int i) { if (enabled && transactionCounter != null) @@ -331,21 +375,39 @@ public void logPerformance() { long currentRows = (long) rowEventCounter.get(); long currentTxns = (long) transactionCounter.get(); + long currentDebezium = (long) debeziumEventCounter.get(); + long currentSerdRows = (long) serdRowRecordCounter.get(); + long currentSerdTxs = (long) serdTxRecordCounter.get(); long deltaRows = currentRows - lastRowChangeCount; long deltaTxns = currentTxns - lastTransactionCount; + long deltaDebezium = currentDebezium - lastDebeziumCount; + long deltaSerdRows = currentSerdRows - lastSerdRowRecordCount; + long deltaSerdTxs = currentSerdTxs - lastSerdTxRecordCount; lastRowChangeCount = currentRows; lastTransactionCount = currentTxns; + lastDebeziumCount = currentDebezium; + lastSerdRowRecordCount = currentSerdRows; + lastSerdTxRecordCount = currentSerdTxs; double seconds = monitorReportInterval / 1000.0; + double rowOips = deltaRows / seconds; double txnOips = deltaTxns / seconds; + double dbOips = deltaDebezium / seconds; + double serdRowsOips = deltaSerdRows / seconds; + double serdTxsOips = deltaSerdTxs / seconds; LOGGER.info( - "Performance report: +{} rows (+{}/s), +{} transactions (+{}/s) in {} ms", + "Performance report: +{} rows (+{}/s), +{} transactions (+{}/s), +{} debezium (+{}/s)" + + ", +{} serdRows (+{}/s), +{} serdTxs (+{}/s)" + + " in {} ms", deltaRows, String.format("%.2f", rowOips), deltaTxns, String.format("%.2f", txnOips), + deltaDebezium, String.format("%.2f", dbOips), + deltaSerdRows, String.format("%.2f", serdRowsOips), + deltaSerdTxs, String.format("%.2f", serdTxsOips), monitorReportInterval ); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java index 2ec5995..f20a98d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java @@ -23,17 +23,15 @@ import io.pixelsdb.pixels.common.physical.PhysicalReader; import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.ProtoType; import io.pixelsdb.pixels.sink.event.TableEnginePipelineManager; import io.pixelsdb.pixels.sink.event.TransactionEventEngineProvider; -import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,10 +39,8 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.Map; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -53,15 +49,9 @@ * @author: AntiO2 * @date: 2025/10/5 11:43 */ -public class SinkStorageProcessor implements MainProcessor +public class SinkStorageProcessor extends AbstractSinkStorageProcessor implements MainProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(SinkStorageProcessor.class); - private final AtomicBoolean running = new AtomicBoolean(false); - - private final String topic; - private final String baseDir; - private final EtcdFileRegistry etcdFileRegistry; - private final List files; private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; private final TableEnginePipelineManager tableEnginePipelineManager = new TableEnginePipelineManager(); @@ -69,22 +59,27 @@ public class SinkStorageProcessor implements MainProcessor private final Thread transactionProcessorThread; private final Thread transAdapterThread; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final Map>> queueMap = new ConcurrentHashMap<>(); + private final Map consumerThreads = new ConcurrentHashMap<>(); + private final int maxQueueCapacity = 10000; + - private final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); public SinkStorageProcessor() { - PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - this.topic = pixelsSinkConfig.getSinkProtoData(); - this.baseDir = pixelsSinkConfig.getSinkProtoDir(); - this.etcdFileRegistry = new EtcdFileRegistry(topic, baseDir); - this.files = this.etcdFileRegistry.listAllFiles(); - this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); this.transAdapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); } + @Override + ProtoType getProtoType(int i) + { + return ProtoType.fromInt(i); + } + + private final CompletableFuture POISON_PILL = new CompletableFuture<>(); + @Override public void start() { @@ -98,38 +93,50 @@ public void start() LOGGER.info("Start read from file {}", file); try(PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) { + long offset = 0; + BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); + BlockingQueue> transQueue = new LinkedBlockingQueue<>(); while(true) { try { - int keyLen = reader.readInt(ByteOrder.BIG_ENDIAN); - int valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - ByteBuffer keyBuffer = copyToHeap(reader.readFully(keyLen)).order(ByteOrder.BIG_ENDIAN); - ByteBuffer valueBuffer = copyToHeap(reader.readFully(valueLen)).order(ByteOrder.BIG_ENDIAN); - ProtoType protoType = ProtoType.fromInt(keyBuffer.getInt()); + int keyLen, valueLen; + reader.seek(offset); try { - switch (protoType) - { - case ROW -> { - rowExecutor.submit(() -> { - handleRowChangeSourceRecord(keyBuffer, valueBuffer); - }); - } - case TRANS -> { - transExecutor.submit(() -> { - try - { - handleTransactionSourceRecord(valueBuffer); - } catch (InterruptedException e) - { - throw new RuntimeException(e); - } - }); - } - } - } catch (Exception e) { - LOGGER.error("Error processing record", e); + keyLen = reader.readInt(ByteOrder.BIG_ENDIAN); + valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + } catch (IOException e) { + // EOF + break; } - } catch (IOException e) { + + ByteBuffer keyBuffer = copyToHeap(reader.readFully(keyLen)).order(ByteOrder.BIG_ENDIAN); + ProtoType protoType = getProtoType(keyBuffer.getInt()); + offset += Integer.BYTES * 2 + keyLen; + CompletableFuture valueFuture = reader.readAsync(offset, valueLen) + .thenApply(this::copyToHeap) + .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); + // move offset for next record + offset += valueLen; + + // Compute queue key (for example: schemaName + tableName or protoType) + SchemaTableName queueKey = computeQueueKey(keyBuffer, protoType); + + // Get or create queue + BlockingQueue> queue = + queueMap.computeIfAbsent(queueKey, + k -> new LinkedBlockingQueue<>(maxQueueCapacity)); + + // Put future in queue + queue.put(valueFuture); + + // Start consumer thread if not exists + consumerThreads.computeIfAbsent(queueKey, k -> { + Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); + t.setName("consumer-" + queueKey); + t.start(); + return t; + }); + } catch (IOException | InterruptedException e) { break; } } @@ -139,15 +146,30 @@ public void start() } } - } - private void handleTransactionSourceRecord(ByteBuffer sourceRecord) throws InterruptedException - { - rawTransactionQueue.put(sourceRecord); + // signal all queues to stop + queueMap.values().forEach(q -> { + try { + q.put(POISON_PILL); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + + // wait all consumers to finish + consumerThreads.values().forEach(t -> { + try { + t.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); } - public static ByteBuffer copyToHeap(ByteBuffer directBuffer) { + + + private ByteBuffer copyToHeap(ByteBuffer directBuffer) { ByteBuffer duplicate = directBuffer.duplicate(); ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); heapBuffer.put(duplicate); @@ -160,17 +182,56 @@ private static String readString(ByteBuffer buffer, int len) { return new String(bytes); } + private void consumeQueue(SchemaTableName key, BlockingQueue> queue, ProtoType protoType) { + try { + while (true) { + CompletableFuture value = queue.take(); + if(value == POISON_PILL) + { + break; + } + ByteBuffer valueBuffer = value.get(); + metricsFacade.recordDebeziumEvent(); + switch (protoType) { + case ROW -> handleRowChangeSourceRecord(key, valueBuffer); + case TRANS -> handleTransactionSourceRecord(valueBuffer); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + LOGGER.error("Error in async processing", e); + } + } + static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); + + private SchemaTableName computeQueueKey(ByteBuffer keyBuffer, ProtoType protoType) { + switch (protoType) { + case ROW -> + { + int schemaLen = keyBuffer.getInt(); + int tableLen = keyBuffer.getInt(); + String schemaName = readString(keyBuffer, schemaLen); + String tableName = readString(keyBuffer, tableLen); + return new SchemaTableName(schemaName, tableName); + } + case TRANS -> + { + return transactionSchemaTableName; + } + default -> + { + throw new IllegalArgumentException("Proto type " + protoType.toString()); + } + } + } + + private void handleRowChangeSourceRecord(SchemaTableName schemaTableName, ByteBuffer dataBuffer) + { + tableEnginePipelineManager.routeRecord(schemaTableName, dataBuffer); + } private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBuffer) { - SchemaTableName schemaTableName = null; - { - // CODE BLOCK VERSION 1 - int schemaLen = keyBuffer.getInt(); - int tableLen = keyBuffer.getInt(); - String schemaName = readString(keyBuffer, schemaLen); - String tableName = readString(keyBuffer, tableLen); - schemaTableName = new SchemaTableName(schemaName, tableName); - } { // CODE BLOCK VERSION 2 // long tableId = keyBuffer.getLong(); @@ -183,14 +244,22 @@ private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBu // } } - tableEnginePipelineManager.routeRecord(schemaTableName, dataBuffer); +// tableEnginePipelineManager.routeRecord(schemaTableName, dataBuffer); } private void processTransactionSourceRecord() { while (true) { + ByteBuffer data = null; + try + { + data = rawTransactionQueue.take(); + } catch (InterruptedException e) + { + throw new RuntimeException(e); + } try { - ByteBuffer data = rawTransactionQueue.take(); SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(data); + metricsFacade.recordSerdTxChange(); if (tx != null) { transactionEventProvider.getEventQueue().put(tx); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 20331de..2e7f27a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -43,20 +43,19 @@ public class TableProcessor implements StoppableProcessor, Runnable private Thread processorThread; private final TransactionCoordinator transactionCoordinator; private final TableEventProvider tableEventProvider; - private final SchemaTableName schemaTableName; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - public TableProcessor(TableEventProvider tableEventProvider, SchemaTableName schemaTableName) + public TableProcessor(TableEventProvider tableEventProvider) { this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); this.tableEventProvider = tableEventProvider; - this.schemaTableName = schemaTableName; + } @Override public void run() { - processorThread = new Thread(this::processLoop, "processor-table" + schemaTableName.getTableName()); + processorThread = new Thread(this::processLoop); processorThread.start(); } @@ -79,7 +78,7 @@ private void processLoop() Thread.currentThread().interrupt(); } } - LOGGER.info("Processor thread exited for {}", schemaTableName.toString()); + LOGGER.info("Processor thread exited"); } @Override diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java index b32ac87..d6e34d3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java @@ -22,7 +22,8 @@ public enum PixelsSinkMode { CSV, RETINA, - PROTO; + PROTO, + NONE; public static PixelsSinkMode fromValue(String value) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java index d762334..93044a9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java @@ -59,10 +59,52 @@ public ProtoWriter() throws IOException public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { + byte[] transData = transactionMetadata.toByteArray(); - ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); - buffer.putInt(ProtoType.TRANS.toInt()); - return writeData(buffer.array(), transData); + return writeData(-1, transData); + // ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); +// buffer.putInt(ProtoType.TRANS.toInt()); +// return writeData(buffer.array(), transData); + } + + public boolean write(SinkProto.RowRecord rowRecord) + { + byte[] rowData = rowRecord.toByteArray(); + String tableName = rowRecord.getSource().getTable(); + String schemaName = rowRecord.getSource().getDb(); + + long tableId; + try + { + tableId = instance.getTableId(schemaName, tableName); + } catch (SinkException e) + { + LOGGER.error("Error while getting schema table id.", e); + return false; + } + { + return writeData((int)tableId, rowData); + } + +// ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES + Long.BYTES); +// keyBuffer.putInt(ProtoType.ROW.toInt()) +// .putLong(tableId); + + +// byte[] schemaNameBytes = schemaName.getBytes(); +// byte[] tableNameBytes = tableName.getBytes(); +// +// ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES * 3 + schemaNameBytes.length + tableNameBytes.length); +// keyBuffer.putInt(ProtoType.ROW.toInt()).putInt(schemaNameBytes.length).putInt(tableNameBytes.length); +// keyBuffer.put(schemaNameBytes).put(tableNameBytes); +// return writeData(keyBuffer.array(), rowData); + } + // key: -1 means transaction, else means table id + private boolean writeData(int key, byte[] data) + { + ByteBuffer buf = ByteBuffer.allocate(Integer.BYTES + Integer.BYTES + data.length).order(ByteOrder.BIG_ENDIAN); // key + value len + data + buf.putInt(key).putInt(data.length).put(data); + return writeBuffer(buf); } private boolean writeData(byte[] key, byte[] data) @@ -70,6 +112,11 @@ private boolean writeData(byte[] key, byte[] data) ByteBuffer buf = ByteBuffer.allocate(Integer.BYTES + Integer.BYTES + key.length + data.length).order(ByteOrder.BIG_ENDIAN); // rowLen + type + data buf.putInt(key.length).putInt(data.length).put(key).put(data); + return writeBuffer(buf); + } + + private boolean writeBuffer(ByteBuffer buf) + { PhysicalWriter writer; try { writer = writerManager.current(); @@ -89,34 +136,7 @@ public boolean write(RowChangeEvent rowChangeEvent) return write(rowChangeEvent.getRowRecord()); } - public boolean write(SinkProto.RowRecord rowRecord) - { - byte[] rowData = rowRecord.toByteArray(); - String tableName = rowRecord.getSource().getTable(); - String schemaName = rowRecord.getSource().getDb(); - -// long tableId; -// try -// { -// tableId = instance.getTableId(schemaName, tableName); -// } catch (SinkException e) -// { -// LOGGER.error("Error while getting schema table id.", e); -// return false; -// } -// -// ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES + Long.BYTES); -// keyBuffer.putInt(ProtoType.ROW.toInt()) -// .putLong(tableId); - - byte[] schemaNameBytes = schemaName.getBytes(); - byte[] tableNameBytes = tableName.getBytes(); - ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES * 3 + schemaNameBytes.length + tableNameBytes.length); - keyBuffer.putInt(ProtoType.ROW.toInt()).putInt(schemaNameBytes.length).putInt(tableNameBytes.length); - keyBuffer.put(schemaNameBytes).put(tableNameBytes); - return writeData(keyBuffer.array(), rowData); - } @Override public void flush() diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java index 2f767f1..728604a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java @@ -113,17 +113,18 @@ public void flush() tableUpdateCount.add(smallBatch.size()); } + if(sinkMode == PixelsSinkMode.NONE) + { + // for test reason + updateCtxCounters(txIds, fullTableName, tableUpdateCount); + return; + } + CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); updateRecordResponseCompletableFuture.thenAccept( resp -> { - for(int i = 0; i < txIds.size(); i++) - { - metricsFacade.recordRowEvent(tableUpdateCount.get(i)); - String writeTxId = txIds.get(i); - SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(writeTxId); - sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); - } + updateCtxCounters(txIds, fullTableName, tableUpdateCount); } ); } finally @@ -132,6 +133,17 @@ public void flush() } } + private void updateCtxCounters(List txIds, List fullTableName, List tableUpdateCount) + { + for(int i = 0; i < txIds.size(); i++) + { + metricsFacade.recordRowEvent(tableUpdateCount.get(i)); + String writeTxId = txIds.get(i); + SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(writeTxId); + sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); + } + } + private RetinaProto.TableUpdateData buildTableUpdateDataFromBatch(String txId, List smallBatch) { SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java index 755767d..cec4056 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java @@ -57,6 +57,7 @@ public abstract class TableWriter protected ScheduledFuture flushTask = null; protected String fullTableName; protected PixelsSinkConfig config; + protected PixelsSinkMode sinkMode; private MetricsFacade metricsFacade = MetricsFacade.getInstance(); protected TableWriter(String tableName) throws IOException @@ -65,6 +66,7 @@ protected TableWriter(String tableName) throws IOException this.tableName = tableName; this.delegate = new RetinaWriter(); this.flushInterval = config.getFlushIntervalMs(); + this.sinkMode = config.getPixelsSinkMode(); } protected static final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties index 3d1f5ce..df6f993 100644 --- a/src/main/resources/pixels-sink.aws.properties +++ b/src/main/resources/pixels-sink.aws.properties @@ -1,9 +1,7 @@ -# engine | kafka -sink.datasource=engine - -# embedded engine config - - +# engine | kafka | storage +sink.datasource=storage +# Sink Config: retina | csv | proto | none +sink.mode=proto # Kafka Config bootstrap.servers=realtime-kafka-2:29092 group.id=3078 @@ -15,14 +13,13 @@ value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeseri topic.prefix=postgresql.oltp_server consumer.capture_database=pixels_bench_sf1x consumer.include_tables= -# Sink Config -sink.mode=retina + sink.csv.path=./data sink.csv.enable_header=false ## Retina Config sink.retina.embedded=false # stub or stream - sink.retina.mode=stream +sink.retina.mode=stream #sink.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 @@ -31,6 +28,17 @@ sink.timeout.ms=5000 sink.flush.interval.ms=100 sink.flush.batch.size=200 sink.max.retries=3 + +## sink commit +sink.commit.batch.size=200 +sink.commit.batch.worker=32 +sink.commit.batch.delay=200 + +## Proto Config +sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp +sink.proto.data=data +sink.proto.maxRecords=1000000 + ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config diff --git a/src/main/resources/pixels-sink.local.properties b/src/main/resources/pixels-sink.local.properties index 9b0cf7d..1dc91ac 100644 --- a/src/main/resources/pixels-sink.local.properties +++ b/src/main/resources/pixels-sink.local.properties @@ -29,6 +29,11 @@ sink.flush.interval.ms=100 sink.flush.batch.size=100 sink.max.retries=3 +## sink commit +sink.commit.batch.size=500 +sink.commit.batch.worker=16 +sink.commit.batch.delay=200 + ## Proto Config sink.proto.dir=file:///home/pixels/projects/pixels-sink/tmp sink.proto.data=data From 2c5a6a14fd5c781dc0ea00f8e0f463a6035a908c Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 8 Oct 2025 11:49:17 +0000 Subject: [PATCH 05/53] Add Web Monitor --- develop/index.html | 276 ++++++++++++++++++ perf_web_monitor.py | 73 +++++ .../pixels/sink/config/PixelsSinkConfig.java | 6 +- .../sink/metadata/TableMetadataRegistry.java | 2 +- .../pixels/sink/processor/MetricsFacade.java | 16 + src/main/resources/pixels-sink.aws.properties | 11 +- 6 files changed, 375 insertions(+), 9 deletions(-) create mode 100644 develop/index.html create mode 100644 perf_web_monitor.py diff --git a/develop/index.html b/develop/index.html new file mode 100644 index 0000000..7dedf06 --- /dev/null +++ b/develop/index.html @@ -0,0 +1,276 @@ + + + + + Pixels Sink Performance Dashboard + + + + + + +
+ +
+ +
+
+ + + + diff --git a/perf_web_monitor.py b/perf_web_monitor.py new file mode 100644 index 0000000..3d4675c --- /dev/null +++ b/perf_web_monitor.py @@ -0,0 +1,73 @@ +from flask import Flask, render_template, jsonify, request +import pandas as pd +import os +from functools import lru_cache +from time import time + +# Configuration +DATA_DIR = "/home/ubuntu/pixels-sink/tmp" +PORT = 8083 +CACHE_TTL = 5 # seconds + +app = Flask(__name__, template_folder='develop') + +# ---- 1. 文件读取缓存(减少频繁IO) ---- +@lru_cache(maxsize=64) +def _read_csv_cached(path, mtime): + """Read CSV with simple caching by modification time.""" + df = pd.read_csv(path, names=["time", "rows", "txns", "debezium", "serdRows", "serdTxs"]) + df = df.tail(300) + # drop continuous zero lines + mask = (df.drop(columns=["time"]).sum(axis=1) != 0) + df = df.loc[mask] + return {col: df[col].tolist() for col in df.columns} + +def read_csv_with_cache(path): + """Wrapper that invalidates cache when file changes.""" + try: + mtime = os.path.getmtime(path) + return _read_csv_cached(path, mtime) + except Exception as e: + print(f"[WARN] Failed to read {path}: {e}") + return None + +# ---- 2. 首页 ---- +@app.route('/') +def index(): + return render_template('index.html') + +# ---- 3. 文件列表 ---- +@app.route('/list') +def list_csv(): + try: + files = [ + f for f in os.listdir(DATA_DIR) + if f.endswith(".csv") and os.path.isfile(os.path.join(DATA_DIR, f)) + ] + return jsonify(sorted(files)) + except Exception as e: + print(f"[ERROR] list_csv failed: {e}") + return jsonify([]) + +@app.route('/data/') +def get_data_file(filename): + print(filename) + path = os.path.join(DATA_DIR, filename) + if not os.path.exists(path): + abort(404, description=f"File {filename} not found") + + data = read_csv_with_cache(path) + if not data: + print("Empty") + return jsonify({}) # 文件为空返回空对象 + return jsonify(data) + +# ---- 5. 简单健康检查 ---- +@app.route('/health') +def health(): + return jsonify({"status": "ok", "time": time()}) + +# ---- 6. 启动 ---- +if __name__ == '__main__': + os.makedirs(DATA_DIR, exist_ok=True) + app.run(host='0.0.0.0', port=PORT, debug=False) diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index d93ba60..5b93d43 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -56,9 +56,6 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.remote.port", defaultValue = "9090") private short remotePort; - @ConfigKey(value = "sink.batch.size", defaultValue = "5000") - private int batchSize; - @ConfigKey(value = "sink.timeout.ms", defaultValue = "30000") private int timeoutMs; @@ -86,6 +83,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.report.interval", defaultValue = "5000") private short monitorReportInterval; + @ConfigKey(value = "sink.monitor.report.file", defaultValue = "/tmp/sink.csv") + private String monitorReportFile; + @ConfigKey(value = "sink.rpc.enable", defaultValue = "false") private boolean rpcEnable; diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index 2cd1f2c..cd6b5a8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -144,7 +144,7 @@ private SchemaTableName loadSchemaTableName(long tableId) throws SinkException { schemas = metadataService.getSchemas(); } - Table table = metadataService.getTable(tableId); + Table table = metadataService.getTableById(tableId); long schemaId = table.getSchemaId(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java index 10f45a8..e6b9922 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java @@ -26,6 +26,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.FileWriter; +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import java.util.concurrent.atomic.AtomicBoolean; public class MetricsFacade @@ -54,6 +58,8 @@ public class MetricsFacade private final boolean monitorReportEnabled; private final int monitorReportInterval; + private final String monitorReportPath; + private final AtomicBoolean running = new AtomicBoolean(false); private Thread reportThread; @@ -185,6 +191,7 @@ private MetricsFacade(boolean enabled) monitorReportEnabled = config.isMonitorReportEnabled(); monitorReportInterval = config.getMonitorReportInterval(); + monitorReportPath = config.getMonitorReportFile(); if(monitorReportEnabled) { @@ -410,5 +417,14 @@ public void logPerformance() deltaSerdTxs, String.format("%.2f", serdTxsOips), monitorReportInterval ); + + String time = LocalDateTime.now().format(DateTimeFormatter.ofPattern("HH:mm:ss")); + // Append to CSV for plotting + try (FileWriter fw = new FileWriter(monitorReportPath, true)) { + fw.write(String.format("%s,%.2f,%.2f,%.2f,%.2f,%.2f%n", + time, rowOips, txnOips, dbOips, serdRowsOips, serdTxsOips)); + } catch (IOException e) { + LOGGER.warn("Failed to write perf metrics: " + e.getMessage()); + } } } \ No newline at end of file diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties index df6f993..f77131e 100644 --- a/src/main/resources/pixels-sink.aws.properties +++ b/src/main/resources/pixels-sink.aws.properties @@ -1,7 +1,7 @@ # engine | kafka | storage sink.datasource=storage # Sink Config: retina | csv | proto | none -sink.mode=proto +sink.mode=retina # Kafka Config bootstrap.servers=realtime-kafka-2:29092 group.id=3078 @@ -23,14 +23,13 @@ sink.retina.mode=stream #sink.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 -sink.batch.size=100 sink.timeout.ms=5000 sink.flush.interval.ms=100 -sink.flush.batch.size=200 +sink.flush.batch.size=500 sink.max.retries=3 ## sink commit -sink.commit.batch.size=200 +sink.commit.batch.size=500 sink.commit.batch.worker=32 sink.commit.batch.delay=200 @@ -45,12 +44,14 @@ sink.registry.url=http://localhost:8080/apis/registry/v2 transaction.topic.suffix=transaction #transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer -sink.trans.batch.size=100 +sink.trans.batch.size=200 ## Batch or trans or record sink.trans.mode=batch # Sink Metrics sink.monitor.enable=true sink.monitor.port=9464 +sink.monitor.report.interval=5000 +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/ParallelRocksDB2.csv # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 From 6e67f8d316bf220ac1eeacfbbc2b042fe58aab3b Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 10 Oct 2025 21:02:17 +0800 Subject: [PATCH 06/53] Refactor --- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 34 +- .../pixels/sink/PixelsSinkProvider.java | 29 +- .../concurrent/TransactionCoordinator.java | 466 ------------------ .../sink/concurrent/TransactionState.java | 99 ---- .../pixels/sink/config/ConfigKey.java | 5 +- .../pixels/sink/config/ConfigLoader.java | 63 ++- .../pixels/sink/config/PixelsSinkConfig.java | 14 +- .../sink/config/PixelsSinkDefaultConfig.java | 3 - .../pixels/sink/config/TransactionConfig.java | 2 +- .../pixels/sink/event/RowChangeEvent.java | 7 +- .../event/TableEnginePipelineManager.java | 79 --- .../sink/event/TableEventEngineProvider.java | 90 ---- .../sink/event/TableEventStorageProvider.java | 151 ------ .../sink/event/TablePipelineManager.java | 41 -- .../event/TableStoragePipelineManager.java | 59 --- .../deserializer/DeserializerUtil.java | 36 +- .../RowChangeEventAvroDeserializer.java | 8 +- .../RowChangeEventJsonDeserializer.java | 6 +- .../RowChangeEventStructDeserializer.java | 17 +- .../deserializer/RowDataParser.java | 8 +- .../deserializer/SchemaDeserializer.java | 4 +- .../TransactionAvroMessageDeserializer.java | 4 +- .../TransactionJsonMessageDeserializer.java | 4 +- .../TransactionStructMessageDeserializer.java | 26 +- .../sink/metadata/TableMetadataRegistry.java | 3 +- .../AbstractSinkStorageProcessor.java | 39 -- .../pixels/sink/processor/TableProcessor.java | 38 +- .../pixels/sink/processor/TopicProcessor.java | 13 +- .../sink/processor/TransactionProcessor.java | 34 +- .../pixels/sink/provider/EventProvider.java | 176 +++++++ .../sink/{event => provider}/ProtoType.java | 12 +- .../provider/TableEventEngineProvider.java | 53 ++ .../TableEventKafkaProvider.java} | 83 +--- .../TableEventProvider.java | 16 +- .../provider/TableEventStorageProvider.java | 63 +++ ...leProviderAndProcessorPipelineManager.java | 88 ++++ .../TransactionEventEngineProvider.java | 25 +- .../TransactionEventKafkaProvider.java | 28 +- .../TransactionEventProvider.java | 23 +- .../TransactionEventStorageProvider.java | 43 ++ .../pixelsdb/pixels/sink/sink/CsvWriter.java | 14 +- .../NoneWriter.java} | 35 +- .../pixels/sink/sink/PixelsSinkWriter.java | 8 +- .../sink/sink/PixelsSinkWriterFactory.java | 37 +- .../pixels/sink/sink/ProtoWriter.java | 34 +- .../sink/sink/RotatingWriterManager.java | 5 +- .../RetinaServiceProxy.java} | 122 +---- .../pixels/sink/sink/retina/RetinaWriter.java | 300 +++++++++++ .../retina}/SinkContext.java | 35 +- .../sink/sink/retina/SinkContextManager.java | 139 ++++++ .../sink/{ => retina}/TableCrossTxWriter.java | 103 ++-- .../{ => retina}/TableSingleTxWriter.java | 32 +- .../sink/sink/{ => retina}/TableWriter.java | 76 ++- .../sink/sink/retina/TableWriterProxy.java | 65 +++ .../retina}/TransactionMode.java | 7 +- .../retina/TransactionProxy.java} | 35 +- .../AbstractSinkStorageSource.java} | 210 ++++---- .../sink/source/FasterSinkStorageSource.java | 77 +++ .../LegacySinkStorageSource.java} | 167 +++---- .../{ => source}/PixelsDebeziumConsumer.java | 64 +-- .../SinkEngineSource.java} | 26 +- .../SinkKafkaSource.java} | 10 +- .../SinkSource.java} | 13 +- .../pixels/sink/source/SinkSourceFactory.java | 36 ++ .../pixels/sink/util/EtcdFileRegistry.java | 79 +-- .../{processor => util}/MetricsFacade.java | 54 +- src/main/resources/log4j2.properties | 2 +- src/main/resources/pixels-sink.aws.properties | 9 +- .../resources/pixels-sink.local.properties | 9 +- src/main/resources/pixels-sink.properties | 4 +- .../pixels/sink/DebeziumEngineTest.java | 11 +- ...dinatorTest.java => RetinaWriterTest.java} | 100 +--- .../sink/consumer/AvroConsumerTest.java | 2 +- .../deserializer/RowBatchTest.java | 2 +- .../RowChangeEventDeserializerTest.java | 3 +- .../deserializer/RowDataParserTest.java | 2 +- .../deserializer/SchemaDeserializerTest.java | 3 +- .../pixels/sink/sink/TestProtoWriter.java | 90 ++-- .../pixels/sink/sink/TestRetinaWriter.java | 34 +- .../sink/util/EtcdFileRegistryTest.java | 5 +- src/test/resources/log4j2.properties | 2 +- 81 files changed, 1847 insertions(+), 2206 deletions(-) delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEnginePipelineManager.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/DeserializerUtil.java (86%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowChangeEventAvroDeserializer.java (97%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowChangeEventJsonDeserializer.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowChangeEventStructDeserializer.java (94%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowDataParser.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/SchemaDeserializer.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/TransactionAvroMessageDeserializer.java (96%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/TransactionJsonMessageDeserializer.java (95%) rename src/main/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/TransactionStructMessageDeserializer.java (82%) delete mode 100644 src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java rename src/main/java/io/pixelsdb/pixels/sink/{event => provider}/ProtoType.java (96%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java rename src/main/java/io/pixelsdb/pixels/sink/{processor/TableMonitor.java => provider/TableEventKafkaProvider.java} (59%) rename src/main/java/io/pixelsdb/pixels/sink/{event => provider}/TableEventProvider.java (63%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java rename src/main/java/io/pixelsdb/pixels/sink/{event => provider}/TransactionEventEngineProvider.java (62%) rename src/main/java/io/pixelsdb/pixels/sink/{event => provider}/TransactionEventKafkaProvider.java (80%) rename src/main/java/io/pixelsdb/pixels/sink/{event => provider}/TransactionEventProvider.java (63%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java rename src/main/java/io/pixelsdb/pixels/sink/{concurrent/TransactionCoordinatorFactory.java => sink/NoneWriter.java} (53%) rename src/main/java/io/pixelsdb/pixels/sink/sink/{RetinaWriter.java => retina/RetinaServiceProxy.java} (51%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java rename src/main/java/io/pixelsdb/pixels/sink/{concurrent => sink/retina}/SinkContext.java (87%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java rename src/main/java/io/pixelsdb/pixels/sink/sink/{ => retina}/TableCrossTxWriter.java (60%) rename src/main/java/io/pixelsdb/pixels/sink/sink/{ => retina}/TableSingleTxWriter.java (78%) rename src/main/java/io/pixelsdb/pixels/sink/sink/{ => retina}/TableWriter.java (84%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java rename src/main/java/io/pixelsdb/pixels/sink/{concurrent => sink/retina}/TransactionMode.java (92%) rename src/main/java/io/pixelsdb/pixels/sink/{concurrent/TransactionManager.java => sink/retina/TransactionProxy.java} (90%) rename src/main/java/io/pixelsdb/pixels/sink/{processor/FasterSinkStorageProcessor.java => source/AbstractSinkStorageSource.java} (56%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java rename src/main/java/io/pixelsdb/pixels/sink/{processor/SinkStorageProcessor.java => source/LegacySinkStorageSource.java} (69%) rename src/main/java/io/pixelsdb/pixels/sink/{ => source}/PixelsDebeziumConsumer.java (77%) rename src/main/java/io/pixelsdb/pixels/sink/{processor/SinkEngineProcessor.java => source/SinkEngineSource.java} (84%) rename src/main/java/io/pixelsdb/pixels/sink/{processor/SinkKafkaProcessor.java => source/SinkKafkaSource.java} (88%) rename src/main/java/io/pixelsdb/pixels/sink/{processor/MainProcessor.java => source/SinkSource.java} (75%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java rename src/main/java/io/pixelsdb/pixels/sink/{processor => util}/MetricsFacade.java (95%) rename src/test/java/io/pixelsdb/pixels/sink/concurrent/{TransactionCoordinatorTest.java => RetinaWriterTest.java} (64%) rename src/test/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowBatchTest.java (96%) rename src/test/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowChangeEventDeserializerTest.java (95%) rename src/test/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/RowDataParserTest.java (97%) rename src/test/java/io/pixelsdb/pixels/sink/{ => event}/deserializer/SchemaDeserializerTest.java (97%) diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 42bc21f..99f92bb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -16,14 +16,15 @@ */ package io.pixelsdb.pixels.sink; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; -import io.pixelsdb.pixels.sink.concurrent.TransactionManager; import io.pixelsdb.pixels.sink.config.CommandLineConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.processor.*; -import io.prometheus.client.hotspot.DefaultExports; +import io.pixelsdb.pixels.sink.sink.retina.TransactionProxy; +import io.pixelsdb.pixels.sink.source.SinkSource; +import io.pixelsdb.pixels.sink.source.SinkSourceFactory; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.hotspot.DefaultExports; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,7 +36,7 @@ public class PixelsSinkApp { private static final Logger LOGGER = LoggerFactory.getLogger(PixelsSinkApp.class); - private static MainProcessor mainProcessor; + private static SinkSource sinkSource; private static HTTPServer prometheusHttpServer; @@ -43,11 +44,10 @@ public static void main(String[] args) throws IOException { Runtime.getRuntime().addShutdownHook(new Thread(() -> { - TransactionManager.Instance().close(); - mainProcessor.stopProcessor(); - TransactionCoordinatorFactory.reset(); + TransactionProxy.Instance().close(); + sinkSource.stopProcessor(); LOGGER.info("Pixels Sink Server shutdown complete"); - if(prometheusHttpServer != null) + if (prometheusHttpServer != null) { prometheusHttpServer.close(); } @@ -56,19 +56,7 @@ public static void main(String[] args) throws IOException init(args); PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - if(config.getDataSource().equals("kafka")) - { - mainProcessor = new SinkKafkaProcessor(); - } else if(config.getDataSource().equals("engine")) - { - mainProcessor = new SinkEngineProcessor(); - } else if(config.getDataSource().equals("storage")) - { - mainProcessor = new FasterSinkStorageProcessor(); - } else - { - throw new IllegalStateException("Unsupported data source type: " + config.getDataSource()); - } + sinkSource = SinkSourceFactory.createSinkSource(); try { @@ -82,7 +70,7 @@ public static void main(String[] args) throws IOException throw new RuntimeException(e); } - mainProcessor.start(); + sinkSource.start(); } private static void init(String[] args) throws IOException diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java index ca642e2..057a7d4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java @@ -19,45 +19,32 @@ import io.pixelsdb.pixels.common.sink.SinkProvider; import io.pixelsdb.pixels.common.utils.ConfigFactory; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.processor.MainProcessor; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; -import io.pixelsdb.pixels.sink.processor.SinkEngineProcessor; -import io.pixelsdb.pixels.sink.processor.SinkKafkaProcessor; +import io.pixelsdb.pixels.sink.source.SinkSource; +import io.pixelsdb.pixels.sink.source.SinkSourceFactory; +import io.pixelsdb.pixels.sink.util.MetricsFacade; public class PixelsSinkProvider implements SinkProvider { - private MainProcessor mainProcessor; + private SinkSource sinkSource; public void start(ConfigFactory config) { PixelsSinkConfigFactory.initialize(config); MetricsFacade.getInstance(); - PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - String dataSource = pixelsSinkConfig.getDataSource(); - if(dataSource.equals("kafka")) - { - mainProcessor = new SinkKafkaProcessor(); - } else if(dataSource.equals("engine")) - { - mainProcessor = new SinkEngineProcessor(); - } else - { - throw new IllegalStateException("Unsupported data source type: " + dataSource); - } - mainProcessor.start(); + sinkSource = SinkSourceFactory.createSinkSource(); + sinkSource.start(); } @Override public void shutdown() { - mainProcessor.stopProcessor(); + sinkSource.stopProcessor(); } @Override public boolean isRunning() { - return mainProcessor.isRunning(); + return sinkSource.isRunning(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java deleted file mode 100644 index 03ecbf2..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinator.java +++ /dev/null @@ -1,466 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.concurrent; - -import io.pixelsdb.pixels.common.exception.TransException; -import io.pixelsdb.pixels.common.transaction.TransContext; -import io.pixelsdb.pixels.common.transaction.TransService; -import io.pixelsdb.pixels.retina.RetinaProto; -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriterFactory; -import io.pixelsdb.pixels.sink.sink.TableCrossTxWriter; -import io.pixelsdb.pixels.sink.sink.TableSingleTxWriter; -import io.prometheus.client.Summary; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; - -public class TransactionCoordinator -{ - public static final int INITIAL_CAPACITY = 11; - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionCoordinator.class); - private static final int MAX_ACTIVE_TX = 1000; - final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(); - final ExecutorService dispatchExecutor = Executors.newCachedThreadPool(); - private final PixelsSinkWriter writer; - private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(1024); - private final ScheduledExecutorService timeoutScheduler = - Executors.newSingleThreadScheduledExecutor(); - private final TransactionManager transactionManager = TransactionManager.Instance(); - private final TransService transService; - private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - // private final BlockingQueue nonTxQueue = new LinkedBlockingQueue<>(); - private long TX_TIMEOUT_MS = PixelsSinkConfigFactory.getInstance().getTransactionTimeout(); - - TransactionCoordinator() - { - try - { - this.writer = PixelsSinkWriterFactory.getWriter(); - } catch (IOException e) - { - throw new RuntimeException(e); - } - transService = TransService.Instance(); - // startDispatchWorker(); - startTimeoutChecker(); - } - - public void processTransactionEvent(SinkProto.TransactionMetadata txMeta) throws SinkException - { - if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) - { - handleTxBegin(txMeta); - } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) - { - handleTxEnd(txMeta); - } - } - - public void processRowEvent(RowChangeEvent event) throws SinkException - { - if (event == null) - { - return; - } - - metricsFacade.recordRowChange(event.getTable(), event.getOp()); - event.startLatencyTimer(); - if (event.getTransaction() == null || event.getTransaction().getId().isEmpty()) - { - handleNonTxEvent(event); - return; - } - - String txId = event.getTransaction().getId(); - String table = event.getFullTableName(); - - long collectionOrder = event.getTransaction().getDataCollectionOrder(); - long totalOrder = event.getTransaction().getTotalOrder(); - - LOGGER.debug("Receive event {} {}/{} {}/{} ", event.getOp().toString(), txId, totalOrder, table, collectionOrder); - AtomicBoolean canWrite = new AtomicBoolean(false); - SinkContext ctx = activeTxContexts.compute(txId, (sourceTxId, sinkContext) -> - { - if (sinkContext == null) - { - SinkContext newSinkContext = new SinkContext(sourceTxId); - newSinkContext.bufferOrphanedEvent(event); - return newSinkContext; - } else - { - if (sinkContext.getPixelsTransCtx() == null) - { - sinkContext.bufferOrphanedEvent(event); - return sinkContext; - } - canWrite.set(true); - return sinkContext; - } - }); - if(canWrite.get()) - { - processRowChangeEvent(ctx, event); - } - } - - private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkException - { - // startTrans(txBegin.getId()).get(); - try - { - startTransSync(txBegin.getId()); - } catch (SinkException e) - { - throw new SinkException("Failed to start trans", e); - } - - } - - private void startTransSync(String sourceTxId) throws SinkException - { - activeTxContexts.compute( - sourceTxId, - (k, oldCtx) -> - { - if (oldCtx == null) - { - return new SinkContext(sourceTxId, transactionManager.getTransContext()); - } else - { - oldCtx.getLock().lock(); - try - { - oldCtx.setPixelsTransCtx(transactionManager.getTransContext()); - handleOrphanEvents(oldCtx); - oldCtx.getCond().signalAll(); - } catch (SinkException e) - { - throw new RuntimeException(e); - } finally - { - oldCtx.getLock().unlock(); - } - return oldCtx; - } - } - ); - LOGGER.trace("Begin Tx Sync: {}", sourceTxId); - } - - private void handleOrphanEvents(SinkContext ctx) throws SinkException - { - Queue buffered = ctx.getOrphanEvent(); - - if (buffered != null) - { - for (RowChangeEvent event : buffered) - { - processRowChangeEvent(ctx, event); - } - } - } - - private void handleTxEnd(SinkProto.TransactionMetadata txEnd) - { - String txId = txEnd.getId(); - SinkContext ctx = getSinkContext(txId); - - transactionExecutor.submit(() -> - { - processTxCommit(txEnd, txId, ctx); - } - ); - switch (pixelsSinkConfig.getTransactionMode()) - { - -// case BATCH -> -// { -// processTxCommit(txEnd, txId, ctx); -// } -// case RECORD -> -// { -// transactionExecutor.submit(() -> -// { -// processTxCommit(txEnd, txId, ctx); -// } -// ); -// } - } - } - - private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, SinkContext ctx) - { - LOGGER.trace("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - if (ctx == null) - { - LOGGER.warn("Sink Context is null"); - return; - } - - try - { - try { - ctx.tableCounterLock.lock(); - while (!ctx.isCompleted(txEnd)) - { - LOGGER.debug("TX End Get Lock {}", txId); - LOGGER.debug("Waiting for events in TX {}: {}", txId, - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - ctx.tableCounterCond.await(); - } - } finally - { - ctx.tableCounterLock.unlock(); - } - - - activeTxContexts.remove(txId); - boolean res = true; - if (res) - { - LOGGER.trace("Committed transaction: {}", txId); - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - transactionManager.commitTransAsync(ctx.getPixelsTransCtx()); - } else - { - LOGGER.info("Abort transaction: {}", txId); - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - CompletableFuture.runAsync(() -> - { - try - { - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); - } catch (TransException e) - { - throw new RuntimeException(e); - } - }).whenComplete((v, ex) -> - { - transLatencyTimer.close(); - if (ex != null) - { - LOGGER.error("Rollback failed", ex); - } - }); - } - } catch (InterruptedException e) - { - try - { - LOGGER.info("Catch Exception, Abort transaction: {}", txId); - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); - } catch (TransException ex) - { - LOGGER.error("Failed to abort transaction {}", txId); - ex.printStackTrace(); - LOGGER.error(ex.getMessage()); - throw new RuntimeException(ex); - } - LOGGER.error(e.getMessage()); - LOGGER.error("Failed to commit transaction {}", txId, e); - } - } - - private void processRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException - { - String table = event.getTable(); - event.setTimeStamp(ctx.getTimestamp()); - event.initIndexKey(); - switch (pixelsSinkConfig.getTransactionMode()) - { - case BATCH -> - { - TableCrossTxWriter.getTableWriter(table).write(event, ctx); - } - case TRANS -> - { - TableSingleTxWriter.getTableWriter(table).write(event, ctx); - } - case RECORD -> - { - dispatchImmediately(event, ctx); - } - } - } - - public SinkContext getSinkContext(String txId) - { - return activeTxContexts.get(txId); - } - - protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) - { - dispatchExecutor.execute(() -> - { - try - { - LOGGER.debug("Dispatching [{}] {}.{} (Order: {}/{}) TX: {}", - event.getOp().name(), - event.getDb(), - event.getTable(), - event.getTransaction() != null ? - event.getTransaction().getDataCollectionOrder() : "N/A", - event.getTransaction() != null ? - event.getTransaction().getTotalOrder() : "N/A", - event.getTransaction().getId()); - Summary.Timer writeLatencyTimer = metricsFacade.startWriteLatencyTimer(); - boolean success = writer.write(event); - writeLatencyTimer.close(); - if (success) - { - metricsFacade.recordTotalLatency(event); - metricsFacade.recordRowChange(event.getTable(), event.getOp()); - event.endLatencyTimer(); - } else - { - // TODO retry? - } - } finally - { - if (ctx != null) - { - ctx.updateCounter(event.getFullTableName()); - if (ctx.pendingEvents.decrementAndGet() == 0 && ctx.completed) - { - ctx.completionFuture.complete(null); - } - } - } - }); - } - - private void startTimeoutChecker() - { - timeoutScheduler.scheduleAtFixedRate(() -> - { - activeTxContexts.entrySet().removeIf(entry -> - { - SinkContext ctx = entry.getValue(); - if (ctx.isExpired()) - { - LOGGER.warn("Transaction timeout: {}", entry.getKey()); - return true; - } - return false; - }); - }, 10, 10, TimeUnit.SECONDS); - } - - private void handleNonTxEvent(RowChangeEvent event) throws SinkException - { - switch (pixelsSinkConfig.getTransactionMode()) - { - case BATCH -> - { - SinkContext sinkContext = new SinkContext("-1"); - TransContext transContext = transactionManager.getTransContext(); - sinkContext.setPixelsTransCtx(transContext); - RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder(); - TableSingleTxWriter.addUpdateData(event, builder); - List tableUpdateDataList = List.of(builder.build()); - writer.writeTrans(pixelsSinkConfig.getCaptureDatabase(), tableUpdateDataList, transContext.getTimestamp()); - transactionManager.commitTransAsync(transContext); - } - case RECORD -> - { - dispatchImmediately(event, null); - } - } - } - - public void shutdown() - { - dispatchExecutor.shutdown(); - timeoutScheduler.shutdown(); - } - - public void setTxTimeoutMs(long txTimeoutMs) - { - TX_TIMEOUT_MS = txTimeoutMs; - } - - private static class OrderedEvent - { - final RowChangeEvent event; - final String table; - final long collectionOrder; - final long totalOrder; - - OrderedEvent(RowChangeEvent event, long collectionOrder, long totalOrder) - { - this.event = event; - this.table = event.getFullTableName(); - this.collectionOrder = collectionOrder; - this.totalOrder = totalOrder; - } - - String getTable() - { - return table; - } - - long getCollectionOrder() - { - return collectionOrder; - } - } - - @Deprecated - private static class BufferedEvent - { - final RowChangeEvent event; - final long collectionOrder; - final long totalOrder; - - BufferedEvent(RowChangeEvent event, long collectionOrder, long totalOrder) - { - this.event = event; - this.collectionOrder = collectionOrder; - this.totalOrder = totalOrder; - } - - long getTotalOrder() - { - return totalOrder; - } - } - -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java b/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java deleted file mode 100644 index 4bb3e9e..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionState.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.concurrent; - -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.event.RowChangeEvent; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; - -public class TransactionState -{ - private final String txId; - private final long beginTs; - private final Map receivedCounts = new ConcurrentHashMap<>(); - private final CopyOnWriteArrayList rowEvents = new CopyOnWriteArrayList<>(); - private Map expectedCounts; // update this when receive END message - private volatile boolean endReceived = false; - - public TransactionState(String txId) - { - this.txId = txId; - this.beginTs = System.currentTimeMillis(); - this.expectedCounts = new HashMap<>(); - } - - public void addRowEvent(RowChangeEvent event) - { - rowEvents.add(event); - String table = event.getTable(); - receivedCounts.compute(table, (k, v) -> - { - if (v == null) - { - return new AtomicInteger(1); - } else - { - v.incrementAndGet(); - return v; - } - }); - } - - public boolean isComplete() - { - return endReceived && - expectedCounts.entrySet().stream() - .allMatch(e -> receivedCounts.getOrDefault(e.getKey(), new AtomicInteger(0)).get() >= e.getValue()); - } - - public void markEndReceived() - { - this.endReceived = true; - } - - public boolean isExpired(long timeoutMs) - { - return System.currentTimeMillis() - beginTs > timeoutMs; - } - - public void setExpectedCounts(List dataCollectionList) - { - this.expectedCounts = dataCollectionList.stream() - .collect(Collectors.toMap( - SinkProto.DataCollection::getDataCollection, - SinkProto.DataCollection::getEventCount - )); - } - - public void setExpectedCounts(Map dataCollectionMap) - { - this.expectedCounts = dataCollectionMap; - } - - public List getRowEvents() - { - return rowEvents; - } -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java index 5beced9..61c585d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java @@ -32,8 +32,11 @@ */ @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.FIELD) -public @interface ConfigKey { +public @interface ConfigKey +{ String value(); + String defaultValue() default ""; + Class defaultClass() default Void.class; } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java index 7ea5ec1..f932e45 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java @@ -19,58 +19,77 @@ package io.pixelsdb.pixels.sink.config; -import io.pixelsdb.pixels.sink.concurrent.TransactionMode; import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; -import io.pixelsdb.pixels.sink.sink.RetinaWriter; +import io.pixelsdb.pixels.sink.sink.retina.RetinaServiceProxy; +import io.pixelsdb.pixels.sink.sink.retina.TransactionMode; import java.lang.reflect.Field; import java.util.Properties; -public class ConfigLoader { - public static void load(Properties props, Object target) { - try { +public class ConfigLoader +{ + public static void load(Properties props, Object target) + { + try + { Class clazz = target.getClass(); - for (Field field : clazz.getDeclaredFields()) { + for (Field field : clazz.getDeclaredFields()) + { ConfigKey annotation = field.getAnnotation(ConfigKey.class); - if (annotation != null) { + if (annotation != null) + { String key = annotation.value(); String value = props.getProperty(key); - if (value == null || value.isEmpty()) { - if (!annotation.defaultValue().isEmpty()) { + if (value == null || value.isEmpty()) + { + if (!annotation.defaultValue().isEmpty()) + { value = annotation.defaultValue(); - } else if (annotation.defaultClass() != Void.class) { + } else if (annotation.defaultClass() != Void.class) + { value = annotation.defaultClass().getName(); } } Object parsed = convert(value, field.getType()); field.setAccessible(true); - try { + try + { field.set(target, parsed); - } catch (IllegalAccessException e) { + } catch (IllegalAccessException e) + { throw new RuntimeException("Failed to inject config for " + key, e); } } } - } catch (Exception e) { + } catch (Exception e) + { throw new RuntimeException("Failed to load config", e); } } - private static Object convert(String value, Class type) { - if (type.equals(int.class) || type.equals(Integer.class)) { + private static Object convert(String value, Class type) + { + if (type.equals(int.class) || type.equals(Integer.class)) + { return Integer.parseInt(value); - } else if (type.equals(long.class) || type.equals(Long.class)) { + } else if (type.equals(long.class) || type.equals(Long.class)) + { return Long.parseLong(value); - } else if (type.equals(short.class) || type.equals(Short.class)) { + } else if (type.equals(short.class) || type.equals(Short.class)) + { return Short.parseShort(value); - } else if (type.equals(boolean.class) || type.equals(Boolean.class)) { + } else if (type.equals(boolean.class) || type.equals(Boolean.class)) + { return Boolean.parseBoolean(value); - } else if (type.equals(PixelsSinkMode.class)) { + } else if (type.equals(PixelsSinkMode.class)) + { return PixelsSinkMode.fromValue(value); - } else if (type.equals(TransactionMode.class)) { + } else if (type.equals(TransactionMode.class)) + { return TransactionMode.fromValue(value); - } else if (type.equals(RetinaWriter.RetinaWriteMode.class)) { - return RetinaWriter.RetinaWriteMode.fromValue(value); + } else if (type.equals(RetinaServiceProxy.RetinaWriteMode.class)) + { + return RetinaServiceProxy.RetinaWriteMode.fromValue(value); } return value; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 5b93d43..f8813f6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -17,15 +17,14 @@ package io.pixelsdb.pixels.sink.config; import io.pixelsdb.pixels.common.utils.ConfigFactory; -import io.pixelsdb.pixels.sink.concurrent.TransactionMode; -import io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer; import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; -import io.pixelsdb.pixels.sink.sink.RetinaWriter; +import io.pixelsdb.pixels.sink.sink.retina.RetinaServiceProxy; +import io.pixelsdb.pixels.sink.sink.retina.TransactionMode; import lombok.Getter; import org.apache.kafka.common.serialization.StringDeserializer; import java.io.IOException; -import java.util.Objects; @Getter public class PixelsSinkConfig @@ -39,7 +38,7 @@ public class PixelsSinkConfig private PixelsSinkMode pixelsSinkMode; @ConfigKey(value = "sink.retina.mode", defaultValue = PixelsSinkDefaultConfig.SINK_RETINA_MODE) - private RetinaWriter.RetinaWriteMode retinaWriteMode; + private RetinaServiceProxy.RetinaWriteMode retinaWriteMode; @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) private TransactionMode transactionMode; @@ -51,7 +50,7 @@ public class PixelsSinkConfig private int commitBatchWorkers; @ConfigKey(value = "sink.commit.batch.delay", defaultValue = "200") - private int commitBatchDelay; + private int commitBatchDelay; @ConfigKey(value = "sink.remote.port", defaultValue = "9090") private short remotePort; @@ -164,7 +163,8 @@ public PixelsSinkConfig(ConfigFactory config) ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); } - public String[] getIncludeTables() { + public String[] getIncludeTables() + { return includeTablesRaw.isEmpty() ? new String[0] : includeTablesRaw.split(","); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java index 0567daf..2464383 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java @@ -17,9 +17,6 @@ package io.pixelsdb.pixels.sink.config; -import io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; - public class PixelsSinkDefaultConfig { public static final String DATA_SOURCE = "kafka"; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java index 101e97c..7bf851c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java @@ -17,7 +17,7 @@ package io.pixelsdb.pixels.sink.config; -import io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer; +import io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer; public class TransactionConfig { diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 1360042..7ce2b06 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -25,7 +25,7 @@ import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadata; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.prometheus.client.Summary; import lombok.Getter; import lombok.Setter; @@ -35,7 +35,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import java.util.stream.IntStream; public class RowChangeEvent @@ -135,15 +134,11 @@ private IndexProto.IndexKey generateIndexKey(TableMetadata tableMetadata, Map enginePipelines = new ConcurrentHashMap<>(); - private final Map storagePipelines = new ConcurrentHashMap<>(); - - public void routeRecord(SchemaTableName schemaTableName, SourceRecord record) { - TableEventEngineProvider tableEventEngineProvider = enginePipelines.computeIfAbsent(schemaTableName, - k-> - { - TableEventEngineProvider newProvider = createEnginePipeline(k); - TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> - new TableProcessor(newProvider)); - tableProcessor.run(); - return newProvider; - }); - tableEventEngineProvider.put(record); - } - - public void routeRecord(SchemaTableName schemaTableName, ByteBuffer record) { - TableEventStorageProvider tableEventStorageProvider = storagePipelines.computeIfAbsent(schemaTableName, - k-> - { - TableEventStorageProvider newProvider = createStoragePipeline(k); - TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(schemaTableName, k2 -> - new TableProcessor(newProvider)); - tableProcessor.run(); - return newProvider; - }); - tableEventStorageProvider.put(record); - } - - private TableEventEngineProvider createEnginePipeline(SchemaTableName schemaTableName) { - TableEventEngineProvider pipeline = new TableEventEngineProvider(schemaTableName); - pipeline.start(); - return pipeline; - } - - private TableEventStorageProvider createStoragePipeline(SchemaTableName schemaTableName) { - TableEventStorageProvider pipeline = new TableEventStorageProvider(schemaTableName); - pipeline.start(); - return pipeline; - } -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java deleted file mode 100644 index 7e273e5..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventEngineProvider.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - - -package io.pixelsdb.pixels.sink.event; - - -import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; -import io.pixelsdb.pixels.sink.exception.SinkException; -import org.apache.kafka.connect.source.SourceRecord; - -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.logging.Logger; - -/** - * @package: io.pixelsdb.pixels.sink.event - * @className: TableEventEngineProvider - * @author: AntiO2 - * @date: 2025/9/26 10:45 - */ -public class TableEventEngineProvider implements TableEventProvider { - private final Logger LOGGER = Logger.getLogger(TableEventEngineProvider.class.getName()); - private final SchemaTableName schemaTableName; - - private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(10000); - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); - private final Thread processorThread; - - public TableEventEngineProvider(SchemaTableName schemaTableName) - { - this.schemaTableName = schemaTableName; - this.processorThread = new Thread(this::processLoop, "TableEventEngineProvider-" + schemaTableName.getTableName()); - } - - @Override - public BlockingQueue getSourceEventQueue() - { - return eventQueue; - } - - public void start() { - processorThread.start(); - } - - public void put(SourceRecord record) { - try { - rawEventQueue.put(record); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - - private void processLoop() { - while (true) { - try { - SourceRecord record = rawEventQueue.take(); - RowChangeEvent rowChangeEvent = null; - try - { - rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(record); - } catch (SinkException e) - { - LOGGER.warning(e.getMessage()); - continue; - } - eventQueue.put(rowChangeEvent); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } - } - } - -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java deleted file mode 100644 index 5476ce7..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventStorageProvider.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - - -package io.pixelsdb.pixels.sink.event; - - -import com.google.protobuf.InvalidProtocolBufferException; -import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; -import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; -import org.apache.kafka.connect.source.SourceRecord; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.*; -import java.util.logging.Logger; - -/** - * @package: io.pixelsdb.pixels.sink.event - * @className: TableEventStorageProvider - * @author: AntiO2 - * @date: 2025/9/26 10:45 - */ -public class TableEventStorageProvider implements TableEventProvider { - private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); - private final SchemaTableName schemaTableName; - - private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(10000); - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); - private final Thread processorThread; - - private final MetricsFacade metricsFacade; - - private static final int BATCH_SIZE = 64; - private static final int THREAD_NUM = 4; - private static final long MAX_WAIT_MS = 5; // configurable - private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); - - public TableEventStorageProvider(SchemaTableName schemaTableName) - { - this.schemaTableName = schemaTableName; - this.processorThread = new Thread(this::processLoop, "TableEventStorageProvider-" + schemaTableName.getTableName()); - this.metricsFacade = MetricsFacade.getInstance(); - } - - public TableEventStorageProvider(int tableId) - { - this.schemaTableName = null; - this.processorThread = new Thread(this::processLoop, "TableEventStorageProvider-" + tableId); - this.metricsFacade = MetricsFacade.getInstance(); - } - - @Override - public BlockingQueue getSourceEventQueue() - { - return eventQueue; - } - - public void start() { - processorThread.start(); - } - - public void put(ByteBuffer record) { - try { - rawEventQueue.put(record); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - - private void processLoop() { - List batch = new ArrayList<>(BATCH_SIZE); - - while (true) { - try { - batch.clear(); - - // take first element (blocking) - ByteBuffer first = rawEventQueue.take(); - batch.add(first); - long startTime = System.nanoTime(); - - // keep polling until batch full or timeout - while (batch.size() < BATCH_SIZE) { - long elapsedMs = (System.nanoTime() - startTime) / 1_000_000; - long remainingMs = MAX_WAIT_MS - elapsedMs; - if (remainingMs <= 0) { - break; - } - - ByteBuffer next = rawEventQueue.poll(remainingMs, TimeUnit.MILLISECONDS); - if (next == null) { - break; - } - batch.add(next); - } - - // parallel decode - List> futures = new ArrayList<>(batch.size()); - for (ByteBuffer data : batch) { - futures.add(decodeExecutor.submit(() -> { - try { - SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(data); - return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } catch (SinkException e) { - LOGGER.warning(e.getMessage()); - return null; - } - })); - } - - // ordered put into queue - for (Future future : futures) { - try { - RowChangeEvent event = future.get(); - if (event != null) { - metricsFacade.recordSerdRowChange(); - eventQueue.put(event); - } - } catch (ExecutionException e) { - LOGGER.warning("Decode failed: " + e.getCause()); - } - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } - } - } - -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java deleted file mode 100644 index e2aad06..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TablePipelineManager.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package io.pixelsdb.pixels.sink.event; - - -import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.sink.processor.TableMonitor; -import io.pixelsdb.pixels.sink.processor.TableProcessor; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -/** - * @package: io.pixelsdb.pixels.sink.event - * @className: TablePipelineManager - * @author: AntiO2 - * @date: 2025/9/26 10:44 - */ -abstract public class TablePipelineManager -{ - protected final ExecutorService executorService = Executors.newCachedThreadPool(); - protected final Map activeTableProcessors = new ConcurrentHashMap<>(); - protected final Map id2activeTableProcessors = new ConcurrentHashMap<>(); -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java deleted file mode 100644 index 908d639..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableStoragePipelineManager.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - - -package io.pixelsdb.pixels.sink.event; - - -import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.sink.processor.TableProcessor; -import org.apache.kafka.connect.source.SourceRecord; - -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * @package: io.pixelsdb.pixels.sink.event - * @className: TableEnginePipelineManager - * @author: AntiO2 - * @date: 2025/9/26 10:44 - */ -public class TableStoragePipelineManager extends TablePipelineManager -{ - private final Map storagePipelines = new ConcurrentHashMap<>(); - - - public void routeRecord(Integer key, ByteBuffer record) { - TableEventStorageProvider tableEventStorageProvider = storagePipelines.computeIfAbsent(key, - k-> - { - TableEventStorageProvider newProvider = createStoragePipeline(k); - TableProcessor tableProcessor = id2activeTableProcessors.computeIfAbsent(k, k2 -> - new TableProcessor(newProvider)); - tableProcessor.run(); - return newProvider; - }); - tableEventStorageProvider.put(record); - } - - private TableEventStorageProvider createStoragePipeline(int key) { - TableEventStorageProvider pipeline = new TableEventStorageProvider(key); - pipeline.start(); - return pipeline; - } -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java similarity index 86% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java index 46538d0..50bb42c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/DeserializerUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java @@ -15,20 +15,13 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; -import com.google.protobuf.ByteString; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.exception.SinkException; -import org.apache.avro.AvroRuntimeException; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; -import java.util.Arrays; - public class DeserializerUtil { static public SinkProto.TransactionStatus getStatusSafely(T record, String field) @@ -46,32 +39,41 @@ static public SinkProto.TransactionStatus getStatusSafely(T record, String f return SinkProto.TransactionStatus.UNRECOGNIZED; } - public static Object getFieldSafely(T record, String field) { - try { - if (record instanceof GenericRecord avro) { + public static Object getFieldSafely(T record, String field) + { + try + { + if (record instanceof GenericRecord avro) + { return avro.get(field); - } else if (record instanceof Struct struct) { + } else if (record instanceof Struct struct) + { return struct.get(field); - } else if (record instanceof SourceRecord sourceRecord) { + } else if (record instanceof SourceRecord sourceRecord) + { return ((Struct) sourceRecord.value()).get(field); } - } catch (Exception e) { + } catch (Exception e) + { return null; } return null; } - public static String getStringSafely(T record, String field) { + public static String getStringSafely(T record, String field) + { Object value = getFieldSafely(record, field); return value != null ? value.toString() : ""; } - public static Long getLongSafely(T record, String field) { + public static Long getLongSafely(T record, String field) + { Object value = getFieldSafely(record, field); return value instanceof Number ? ((Number) value).longValue() : 0L; } - public static Integer getIntSafely(T record, String field) { + public static Integer getIntSafely(T record, String field) + { Object value = getFieldSafely(record, field); return value instanceof Number ? ((Number) value).intValue() : 0; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java similarity index 97% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java index bae9000..0f97664 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventAvroDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import io.apicurio.registry.serde.SerdeConfig; import io.apicurio.registry.serde.avro.AvroKafkaDeserializer; @@ -26,7 +26,7 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.common.serialization.Deserializer; @@ -80,7 +80,7 @@ private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema SinkProto.RowRecord.Builder recordBuilder = SinkProto.RowRecord.newBuilder() .setOp(op) // .setTsMs(DeserializerUtil.getLongSafely(avroRecord, "ts_ms")); -; + ; if (avroRecord.get("source") != null) { //TODO: 这里看下怎么处理,如果没有source信息,其实可以通过topic推出schema和table信息。 @@ -139,7 +139,7 @@ private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder builder .setDb(DeserializerUtil.getStringSafely(source, "db")) - .setSchema(DeserializerUtil.getStringSafely(source, "schema")) + .setSchema(DeserializerUtil.getStringSafely(source, "schema")) .setTable(DeserializerUtil.getStringSafely(source, "table")) // .setVersion(DeserializerUtil.getStringSafely(source, "version")) // .setConnector(DeserializerUtil.getStringSafely(source, "connector")) diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java index e0a005d..94b126f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventJsonDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -24,7 +24,7 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,7 +154,7 @@ private SinkProto.SourceInfo.Builder parseSourceInfo(JsonNode sourceNode) // .setTxId(sourceNode.path("txId").asLong()) // .setLsn(sourceNode.path("lsn").asLong()) // .setXmin(sourceNode.path("xmin").asLong()) - ; + ; } private SinkProto.TransactionInfo parseTransactionInfo(JsonNode txNode) diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java similarity index 94% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java index 6cc0a0b..0780a3c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventStructDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java @@ -16,10 +16,9 @@ */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; -import com.fasterxml.jackson.databind.JsonNode; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; @@ -59,7 +58,7 @@ public static RowChangeEvent convertToRowChangeEvent(SinkProto.RowRecord rowReco } private static RowChangeEvent buildRowRecord(Struct value, - SinkProto.OperationType opType) throws SinkException + SinkProto.OperationType opType) throws SinkException { SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); @@ -68,7 +67,8 @@ private static RowChangeEvent buildRowRecord(Struct value, String schemaName; String tableName; - try { + try + { Struct source = value.getStruct("source"); SinkProto.SourceInfo.Builder sourceInfoBuilder = parseSourceInfo(source); schemaName = sourceInfoBuilder.getDb(); // Notice we use the schema @@ -83,7 +83,8 @@ private static RowChangeEvent buildRowRecord(Struct value, TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); RowDataParser rowDataParser = new RowDataParser(typeDescription); - try { + try + { Struct transaction = value.getStruct("transaction"); SinkProto.TransactionInfo transactionInfo = parseTransactionInfo(transaction); builder.setTransaction(transactionInfo); @@ -111,7 +112,8 @@ private static RowChangeEvent buildRowRecord(Struct value, return event; } - private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) { + private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) + { return SinkProto.SourceInfo.newBuilder() // .setVersion(DeserializerUtil.getStringSafely(source, "version")) // .setConnector(DeserializerUtil.getStringSafely(source, "connector")) @@ -129,7 +131,8 @@ private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) { // .setXmin(DeserializerUtil.getLongSafely(source, "xmin")); } - private static SinkProto.TransactionInfo parseTransactionInfo(T txNode) { + private static SinkProto.TransactionInfo parseTransactionInfo(T txNode) + { return SinkProto.TransactionInfo.newBuilder() .setId(DeserializerUtil.getTransIdPrefix( DeserializerUtil.getStringSafely(txNode, "id"))) diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java index db239f9..7d719a9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/RowDataParser.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java @@ -15,14 +15,12 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; import com.google.protobuf.ByteString; -import io.pixelsdb.pixels.core.PixelsProto; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.util.DateUtil; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; @@ -36,7 +34,6 @@ import java.time.LocalDate; import java.util.Base64; import java.util.Collections; -import java.util.Date; import java.util.Map; class RowDataParser @@ -80,6 +77,7 @@ public void parse(Struct record, SinkProto.RowValue.Builder builder) builder.addValues(parseValue(record.get(fieldName), fieldName, fieldType).build()); } } + private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fieldName, TypeDescription type) { if (valueNode == null || valueNode.isNull()) @@ -112,7 +110,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel case CHAR: { String text = valueNode.asText(); - byte[] bytes = new byte[] { (byte) text.charAt(0) }; + byte[] bytes = new byte[]{(byte) text.charAt(0)}; columnValueBuilder.setValue(ByteString.copyFrom(bytes)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder() // .setKind(PixelsProto.Type.Kind.STRING)); diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java index df66bb3..d5d8b79 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; import io.pixelsdb.pixels.core.TypeDescription; @@ -51,7 +51,7 @@ private static JsonNode findSchemaField(JsonNode schemaNode, String targetField) return null; } - static TypeDescription parseStruct(JsonNode fields) + public static TypeDescription parseStruct(JsonNode fields) { TypeDescription structType = TypeDescription.createStruct(); fields.forEach(field -> diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java similarity index 96% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java index 72d6b24..16f1852 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionAvroMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java @@ -15,14 +15,14 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import io.apicurio.registry.serde.SerdeConfig; import io.apicurio.registry.serde.avro.AvroKafkaDeserializer; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java similarity index 95% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java index 8ce20d9..d7eca6b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionJsonMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java @@ -15,12 +15,12 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.protobuf.util.JsonFormat; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java similarity index 82% rename from src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java rename to src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java index 358a8f3..92d517f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/deserializer/TransactionStructMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java @@ -16,22 +16,15 @@ */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.protobuf.util.JsonFormat; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Map; - /** * @package: io.pixelsdb.pixels.sink.deserializer * @className: TransactionStructMessageDeserializer @@ -43,26 +36,31 @@ public class TransactionStructMessageDeserializer private static final Logger LOGGER = LoggerFactory.getLogger(TransactionStructMessageDeserializer.class); @SuppressWarnings("unchecked") - public static SinkProto.TransactionMetadata convertToTransactionMetadata(T record) { + public static SinkProto.TransactionMetadata convertToTransactionMetadata(T record) + { SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); builder.setStatus(DeserializerUtil.getStatusSafely(record, "status")) .setId(DeserializerUtil.getTransIdPrefix( - DeserializerUtil.getStringSafely(record, "id"))) + DeserializerUtil.getStringSafely(record, "id"))) .setEventCount(DeserializerUtil.getLongSafely(record, "event_count")) .setTimestamp(DeserializerUtil.getLongSafely(record, "ts_ms")); Object collections = DeserializerUtil.getFieldSafely(record, "data_collections"); - if (collections instanceof Iterable) { - for (Object item : (Iterable) collections) { - if (item instanceof GenericRecord collectionRecord) { + if (collections instanceof Iterable) + { + for (Object item : (Iterable) collections) + { + if (item instanceof GenericRecord collectionRecord) + { SinkProto.DataCollection.Builder collectionBuilder = SinkProto.DataCollection.newBuilder(); collectionBuilder.setDataCollection( DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); collectionBuilder.setEventCount( DeserializerUtil.getLongSafely(collectionRecord, "event_count")); builder.addDataCollections(collectionBuilder); - } else if (item instanceof Struct collectionRecord) { + } else if (item instanceof Struct collectionRecord) + { SinkProto.DataCollection.Builder collectionBuilder = SinkProto.DataCollection.newBuilder(); collectionBuilder.setDataCollection( DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index cd6b5a8..daac3ef 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -43,6 +43,7 @@ public class TableMetadataRegistry private final ConcurrentMap registry = new ConcurrentHashMap<>(); private final ConcurrentMap tableId2SchemaTableName = new ConcurrentHashMap<>(); private List schemas; + private TableMetadataRegistry() { } @@ -140,7 +141,7 @@ private SchemaTableName loadSchemaTableName(long tableId) throws SinkException // metadataService try { - if(schemas == null) + if (schemas == null) { schemas = metadataService.getSchemas(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java deleted file mode 100644 index 608d019..0000000 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/AbstractSinkStorageProcessor.java +++ /dev/null @@ -1,39 +0,0 @@ -package io.pixelsdb.pixels.sink.processor; - -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.event.ProtoType; -import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; - -public abstract class AbstractSinkStorageProcessor -{ - protected final AtomicBoolean running = new AtomicBoolean(false); - - protected final String topic; - protected final String baseDir; - protected final EtcdFileRegistry etcdFileRegistry; - protected final List files; - - protected final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); - - protected AbstractSinkStorageProcessor() - { - PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - this.topic = pixelsSinkConfig.getSinkProtoData(); - this.baseDir = pixelsSinkConfig.getSinkProtoDir(); - this.etcdFileRegistry = new EtcdFileRegistry(topic, baseDir); - this.files = this.etcdFileRegistry.listAllFiles(); - } - protected void handleTransactionSourceRecord(ByteBuffer sourceRecord) throws InterruptedException - { - rawTransactionQueue.put(sourceRecord); - } - - abstract ProtoType getProtoType(int i); -} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 2e7f27a..4344f22 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -19,15 +19,15 @@ package io.pixelsdb.pixels.sink.processor; -import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.event.TableEventProvider; -import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.provider.TableEventProvider; +import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; +import io.pixelsdb.pixels.sink.sink.PixelsSinkWriterFactory; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -40,16 +40,15 @@ public class TableProcessor implements StoppableProcessor, Runnable { private static final Logger LOGGER = LoggerFactory.getLogger(TableProcessor.class); private final AtomicBoolean running = new AtomicBoolean(true); - private Thread processorThread; - private final TransactionCoordinator transactionCoordinator; - private final TableEventProvider tableEventProvider; + private final PixelsSinkWriter pixelsSinkWriter; + private final TableEventProvider tableEventProvider; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private Thread processorThread; - public TableProcessor(TableEventProvider tableEventProvider) + public TableProcessor(TableEventProvider tableEventProvider) { - this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); + this.pixelsSinkWriter = PixelsSinkWriterFactory.getWriter(); this.tableEventProvider = tableEventProvider; - } @Override @@ -63,20 +62,12 @@ private void processLoop() { while (running.get()) { - try - { - RowChangeEvent event = tableEventProvider.getSourceEventQueue().take(); - try - { - transactionCoordinator.processRowEvent(event); - } catch (SinkException e) - { - throw new RuntimeException(e); - } - } catch (InterruptedException e) + RowChangeEvent event = tableEventProvider.getRowChangeEvent(); + if (event == null) { - Thread.currentThread().interrupt(); + continue; } + pixelsSinkWriter.writeRow(event); } LOGGER.info("Processor thread exited"); } @@ -86,5 +77,6 @@ public void stopProcessor() { LOGGER.info("Stopping transaction monitor"); running.set(false); + processorThread.interrupt(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java index d72d10d..1f4f6da 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java @@ -18,7 +18,7 @@ package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.event.TablePipelineManager; +import io.pixelsdb.pixels.sink.provider.TableEventKafkaProvider; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ListTopicsResult; @@ -32,7 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -public class TopicProcessor extends TablePipelineManager implements StoppableProcessor, Runnable +public class TopicProcessor implements StoppableProcessor, Runnable { private static final Logger log = LoggerFactory.getLogger(TopicProcessor.class); @@ -44,9 +44,10 @@ public class TopicProcessor extends TablePipelineManager implements StoppablePro private final String baseTopic; private final AtomicBoolean running = new AtomicBoolean(true); - private final Map activeTasks = new ConcurrentHashMap<>(); // track row event consumer + private final Map activeTasks = new ConcurrentHashMap<>(); // track row event consumer private AdminClient adminClient; private Timer timer; + private final ExecutorService executorService = Executors.newCachedThreadPool(); public TopicProcessor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) { @@ -94,7 +95,7 @@ private void shutdownConsumerTasks() activeTasks.forEach((topic, task) -> { log.info("Stopping consumer for topic: {}", topic); - task.shutdown(); + task.close(); }); activeTasks.clear(); } @@ -205,7 +206,7 @@ private void launchConsumerTask(String topic) { try { - TableMonitor task = new TableMonitor(kafkaProperties, topic); + TableEventKafkaProvider task = new TableEventKafkaProvider(kafkaProperties, topic); executorService.submit(task); } catch (Exception e) { @@ -258,7 +259,7 @@ private void handleNewTopics(Set newTopics) { try { - TableMonitor task = new TableMonitor(kafkaProperties, topic); + TableEventKafkaProvider task = new TableEventKafkaProvider(kafkaProperties, topic); executorService.submit(task); activeTasks.put(topic, task); subscribedTopics.add(topic); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java index 3134be4..8b2df64 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -19,31 +19,25 @@ package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.event.TransactionEventProvider; -import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.provider.TransactionEventProvider; +import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; +import io.pixelsdb.pixels.sink.sink.PixelsSinkWriterFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Properties; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; public class TransactionProcessor implements Runnable, StoppableProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(TransactionProcessor.class); - private final TransactionCoordinator transactionCoordinator; + private final PixelsSinkWriter sinkWriter; private final AtomicBoolean running = new AtomicBoolean(true); private final TransactionEventProvider transactionEventProvider; public TransactionProcessor(TransactionEventProvider transactionEventProvider) { this.transactionEventProvider = transactionEventProvider; - this.transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); + this.sinkWriter = PixelsSinkWriterFactory.getWriter(); } @Override @@ -51,21 +45,9 @@ public void run() { while (running.get()) { - try - { - SinkProto.TransactionMetadata transaction = transactionEventProvider.getEventQueue().take(); - try - { - LOGGER.trace("Processing transaction event: {}", transaction.getId()); - transactionCoordinator.processTransactionEvent(transaction); - } catch (SinkException e) - { - throw new RuntimeException(e); - } - } catch (InterruptedException e) - { - Thread.currentThread().interrupt(); - } + SinkProto.TransactionMetadata transaction = transactionEventProvider.getTransaction(); + LOGGER.trace("Processing transaction event: {}", transaction.getId()); + sinkWriter.writeTrans(transaction); } LOGGER.info("Processor thread exited for transaction"); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java new file mode 100644 index 0000000..e19cd9b --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -0,0 +1,176 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.provider; + +import io.pixelsdb.pixels.sink.source.FasterSinkStorageSource; +import io.pixelsdb.pixels.sink.util.MetricsFacade; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.*; + +public abstract class EventProvider implements Runnable, Closeable +{ + private static final Logger LOGGER = LoggerFactory.getLogger(EventProvider.class); + + private static final int BATCH_SIZE = 64; + private static final int THREAD_NUM = 4; + private static final long MAX_WAIT_MS = 5; // configurable + + protected final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(10000); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); + + @Override + public void run() + { + processLoop(); + } + + @Override + public void close() + { + decodeExecutor.shutdown(); + } + + protected void processLoop() + { + List sourceBatch = new ArrayList<>(BATCH_SIZE); + while (true) + { + try + { + sourceBatch.clear(); + // take first element (blocking) + SOURCE_RECORD_T first = getRawEvent(); + sourceBatch.add(first); + long startTime = System.nanoTime(); + + // keep polling until sourceBatch full or timeout + while (sourceBatch.size() < BATCH_SIZE) + { + long elapsedMs = (System.nanoTime() - startTime) / 1_000_000; + long remainingMs = MAX_WAIT_MS - elapsedMs; + if (remainingMs <= 0) + { + break; + } + + SOURCE_RECORD_T next = pollRawEvent(remainingMs); + if (next == null) + { + break; + } + sourceBatch.add(next); + } + + // parallel decode + List> futures = new ArrayList<>(sourceBatch.size()); + for (SOURCE_RECORD_T data : sourceBatch) + { + futures.add(decodeExecutor.submit(() -> + convertToTargetRecord(data))); + } + + // ordered put into queue + for (Future future : futures) + { + try + { + TARGET_RECORD_T event = future.get(); + if (event != null) + { + metricsFacade.recordSerdRowChange(); + putTargetEvent(event); + } + } catch (ExecutionException e) + { + LOGGER.warn("Decode failed: " + e.getCause()); + } + } + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + break; + } + } + } + + abstract TARGET_RECORD_T convertToTargetRecord(SOURCE_RECORD_T record); + + protected TARGET_RECORD_T getTargetEvent() + { + try + { + return eventQueue.take(); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + return null; + } + + protected void putTargetEvent(TARGET_RECORD_T event) + { + try + { + eventQueue.put(event); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + } + + void putRawEvent(SOURCE_RECORD_T record) + { + try + { + rawEventQueue.put(record); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + } + + protected SOURCE_RECORD_T getRawEvent() + { + try + { + return rawEventQueue.take(); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + return null; + } + } + + protected SOURCE_RECORD_T pollRawEvent(long remainingMs) + { + try + { + return rawEventQueue.poll(remainingMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) + { + return null; + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java similarity index 96% rename from src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java rename to src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java index f7c4e12..362fc07 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/ProtoType.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.event; +package io.pixelsdb.pixels.sink.provider; /** @@ -37,11 +37,6 @@ public enum ProtoType this.value = value; } - public int toInt() - { - return value; - } - public static ProtoType fromInt(int value) { for (ProtoType type : ProtoType.values()) @@ -53,4 +48,9 @@ public static ProtoType fromInt(int value) } throw new IllegalArgumentException("Unknown ProtoType value: " + value); } + + public int toInt() + { + return value; + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java new file mode 100644 index 0000000..519a2f7 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java @@ -0,0 +1,53 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.provider; + + +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEventEngineProvider + * @author: AntiO2 + * @date: 2025/9/26 10:45 + */ +public class TableEventEngineProvider extends TableEventProvider +{ + private final Logger LOGGER = LoggerFactory.getLogger(TableEventEngineProvider.class.getName()); + + @Override + RowChangeEvent convertToTargetRecord(T record) + { + SourceRecord sourceRecord = (SourceRecord) record; + try + { + return RowChangeEventStructDeserializer.convertToRowChangeEvent(sourceRecord); + } catch (SinkException e) + { + LOGGER.warn("Failed to convert RowChangeEvent to RowChangeEventStruct {}", e.getMessage()); + return null; + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableMonitor.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java similarity index 59% rename from src/main/java/io/pixelsdb/pixels/sink/processor/TableMonitor.java rename to src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java index 9740336..6aa68fb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableMonitor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java @@ -16,14 +16,11 @@ * */ -package io.pixelsdb.pixels.sink.processor; +package io.pixelsdb.pixels.sink.provider; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.exception.SinkException; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -36,23 +33,18 @@ import java.time.Duration; import java.util.Collections; import java.util.Properties; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; -public class TableMonitor implements Runnable +public class TableEventKafkaProvider extends TableEventProvider { - private static final Logger log = LoggerFactory.getLogger(TableMonitor.class); - private static final TransactionCoordinator transactionCoordinator = TransactionCoordinatorFactory.getCoordinator(); + private static final Logger log = LoggerFactory.getLogger(TableEventKafkaProvider.class); private final Properties kafkaProperties; private final String topic; private final AtomicBoolean running = new AtomicBoolean(true); private final String tableName; - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); private KafkaConsumer consumer; - private Thread processorThread; - public TableMonitor(Properties kafkaProperties, String topic) throws IOException + public TableEventKafkaProvider(Properties kafkaProperties, String topic) throws IOException { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.kafkaProperties = kafkaProperties; @@ -64,16 +56,13 @@ public TableMonitor(Properties kafkaProperties, String topic) throws IOException } @Override - public void run() + protected void processLoop() { try { consumer = new KafkaConsumer<>(kafkaProperties); consumer.subscribe(Collections.singleton(topic)); - processorThread = new Thread(this::processLoop, "processor-" + tableName); - processorThread.start(); - while (running.get()) { try @@ -88,18 +77,14 @@ public void run() { return; } - try - { - eventQueue.put(record.value()); - } catch (InterruptedException e) - { - Thread.currentThread().interrupt(); - } + metricsFacade.recordSerdRowChange(); + putRowChangeEvent(record.value()); }); } } catch (InterruptException ignored) { - + Thread.currentThread().interrupt(); + break; } } } catch (WakeupException e) @@ -107,7 +92,6 @@ public void run() log.info("Consumer wakeup triggered for {}", tableName); } catch (Exception e) { - e.printStackTrace(); log.info("Exception: {}", e.getMessage()); } finally { @@ -116,49 +100,6 @@ public void run() consumer.close(Duration.ofSeconds(5)); log.info("Kafka consumer closed for {}", tableName); } - if (processorThread != null) - { - processorThread.interrupt(); - try - { - processorThread.join(); - } catch (InterruptedException ignored) - { - } - } - } - } - - private void processLoop() - { - while (running.get() || !eventQueue.isEmpty()) - { - try - { - RowChangeEvent event = eventQueue.take(); - // log.info("Event received: {}", event); - try - { - transactionCoordinator.processRowEvent(event); - } catch (SinkException e) - { - throw new RuntimeException(e); - } - } catch (InterruptedException e) - { - Thread.currentThread().interrupt(); - } - } - log.info("Processor thread exited for {}", tableName); - } - - public void shutdown() - { - running.set(false); - log.info("Shutting down consumer for table: {}", tableName); - if (consumer != null) - { - consumer.wakeup(); } } @@ -167,4 +108,10 @@ private String extractTableName(String topic) String[] parts = topic.split("\\."); return parts[parts.length - 1]; } + + @Override + RowChangeEvent convertToTargetRecord(Void record) + { + throw new UnsupportedOperationException(); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java similarity index 63% rename from src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java rename to src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java index c57604b..89175d6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TableEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java @@ -15,10 +15,10 @@ * */ -package io.pixelsdb.pixels.sink.event; +package io.pixelsdb.pixels.sink.provider; -import java.util.concurrent.BlockingQueue; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; /** * @package: io.pixelsdb.pixels.sink.event @@ -26,7 +26,15 @@ * @author: AntiO2 * @date: 2025/9/26 07:47 */ -public interface TableEventProvider +public abstract class TableEventProvider extends EventProvider { - BlockingQueue getSourceEventQueue(); + public RowChangeEvent getRowChangeEvent() + { + return getTargetEvent(); + } + + protected void putRowChangeEvent(RowChangeEvent rowChangeEvent) + { + putTargetEvent(rowChangeEvent); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java new file mode 100644 index 0000000..4d912c5 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java @@ -0,0 +1,63 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.provider; + + +import com.google.protobuf.InvalidProtocolBufferException; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.exception.SinkException; + +import java.nio.ByteBuffer; +import java.util.logging.Logger; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableEventStorageProvider + * @author: AntiO2 + * @date: 2025/9/26 10:45 + */ +public class TableEventStorageProvider extends TableEventProvider +{ + private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); + + protected TableEventStorageProvider() + { + super(); + } + + @Override + RowChangeEvent convertToTargetRecord(T record) + { + ByteBuffer sourceRecord = (ByteBuffer) record; + try + { + SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); + return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); + } catch (InvalidProtocolBufferException e) + { + throw new RuntimeException(e); + } catch (SinkException e) + { + LOGGER.warning(e.getMessage()); + return null; + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java new file mode 100644 index 0000000..d668b4a --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -0,0 +1,88 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.provider; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.processor.TableProcessor; +import org.apache.kafka.connect.source.SourceRecord; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * @package: io.pixelsdb.pixels.sink.event + * @className: TableProviderAndProcessorPipelineManager + * @author: AntiO2 + * @date: 2025/9/26 10:44 + */ +public class TableProviderAndProcessorPipelineManager +{ + protected final Map activeTableProcessors = new ConcurrentHashMap<>(); + protected final Map tableIds = new ConcurrentHashMap<>(); + private final Map> tableProviders = new ConcurrentHashMap<>(); + private final AtomicInteger nextTableId = new AtomicInteger(); + + + public void routeRecord(SchemaTableName schemaTableName, SOURCE_RECORD_T record) + { + routeRecord(getTableId(schemaTableName), record); + } + + public void routeRecord(Integer tableId, SOURCE_RECORD_T record) + { + TableEventProvider pipeline = tableProviders.computeIfAbsent(tableId, k -> + { + TableEventProvider newPipeline = createProvider(record); + TableProcessor tableProcessor = activeTableProcessors.computeIfAbsent(tableId, k2 -> + new TableProcessor(newPipeline) + ); + tableProcessor.run(); + return newPipeline; + }); + pipeline.putRawEvent(record); + } + + private TableEventProvider createProvider(SOURCE_RECORD_T record) + { + Class recordType = record.getClass(); + if (recordType == SourceRecord.class) + { + return new TableEventEngineProvider<>(); + } else if (recordType == ByteBuffer.class) + { + return new TableEventStorageProvider<>(); + } else + { + throw new IllegalArgumentException("Unsupported record type: " + recordType.getName()); + } + } + + private Integer getTableId(SchemaTableName schemaTableName) + { + return tableIds.computeIfAbsent(schemaTableName, k -> allocateTableId()); + } + + private Integer allocateTableId() + { + return nextTableId.getAndIncrement(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java similarity index 62% rename from src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java rename to src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java index fc13b92..0ed74fb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java @@ -16,14 +16,15 @@ */ -package io.pixelsdb.pixels.sink.event; +package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.deserializer.TransactionStructMessageDeserializer; +import io.pixelsdb.pixels.sink.event.deserializer.TransactionStructMessageDeserializer; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; +import java.io.IOException; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -33,26 +34,24 @@ * @author: AntiO2 * @date: 2025/9/25 13:20 */ -public class TransactionEventEngineProvider implements TransactionEventProvider { +public class TransactionEventEngineProvider extends TransactionEventProvider +{ - public static final TransactionEventEngineProvider INSTANCE = new TransactionEventEngineProvider(); - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + public static final TransactionEventEngineProvider INSTANCE = new TransactionEventEngineProvider<>(); - private TransactionEventEngineProvider() {} - - public static TransactionEventEngineProvider getInstance() + public static TransactionEventEngineProvider getInstance() { return INSTANCE; } @Override - public BlockingQueue getEventQueue() { - return eventQueue; - } - - public SinkProto.TransactionMetadata convert(SourceRecord sourceRecord) + SinkProto.TransactionMetadata convertToTargetRecord(T record) { + SourceRecord sourceRecord = (SourceRecord) record; Struct value = (Struct) sourceRecord.value(); + metricsFacade.recordSerdTxChange(); return TransactionStructMessageDeserializer.convertToTransactionMetadata(value); } + + } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java similarity index 80% rename from src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java rename to src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java index aa014af..6762168 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.event; +package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.SinkProto; @@ -27,7 +27,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.errors.WakeupException; -import java.io.Closeable; import java.io.IOException; import java.time.Duration; import java.util.Collections; @@ -42,29 +41,23 @@ * @author: AntiO2 * @date: 2025/9/25 13:40 */ -public class TransactionEventKafkaProvider implements TransactionEventProvider, Runnable, Closeable +public class TransactionEventKafkaProvider extends TransactionEventProvider { private final AtomicBoolean running = new AtomicBoolean(true); private final String transactionTopic; private final KafkaConsumer consumer; - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + private TransactionEventKafkaProvider() { Properties kafkaProperties = new Properties(); PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - this.transactionTopic = pixelsSinkConfig.getTopicPrefix() + "." + pixelsSinkConfig.getTransactionTopicSuffix(); this.consumer = new KafkaConsumer<>(kafkaProperties); } - @Override - public BlockingQueue getEventQueue() - { - return eventQueue; - } @Override - public void run() + public void processLoop() { consumer.subscribe(Collections.singletonList(transactionTopic)); while (running.get()) @@ -77,13 +70,11 @@ public void run() for (ConsumerRecord record : records) { - try - { - eventQueue.put(record.value()); - } catch (InterruptedException e) + if(record.value() == null) { - Thread.currentThread().interrupt(); + continue; } + putTargetEvent(record.value()); } } catch (WakeupException e) { @@ -100,8 +91,9 @@ public void run() } @Override - public void close() throws IOException + SinkProto.TransactionMetadata convertToTargetRecord(T record) { - running.set(true); + throw new UnsupportedOperationException(); } + } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java similarity index 63% rename from src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java rename to src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java index 5fb4ae4..7de77c6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/TransactionEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java @@ -15,20 +15,19 @@ * */ -package io.pixelsdb.pixels.sink.event; - +package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.SinkProto; -import java.util.concurrent.BlockingQueue; - -/** - * @package: io.pixelsdb.pixels.sink.event - * @className: TransactionEventProvider - * @author: AntiO2 - * @date: 2025/9/25 13:37 - */ -public interface TransactionEventProvider +public abstract class TransactionEventProvider extends EventProvider { - BlockingQueue getEventQueue(); + public void putTransRawEvent(SOURCE_RECORD_T record) + { + putRawEvent(record); + } + + public SinkProto.TransactionMetadata getTransaction() + { + return getTargetEvent(); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java new file mode 100644 index 0000000..daacd2b --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java @@ -0,0 +1,43 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.provider; + +import com.google.protobuf.InvalidProtocolBufferException; +import io.pixelsdb.pixels.sink.SinkProto; + +import java.nio.ByteBuffer; + +public class TransactionEventStorageProvider extends TransactionEventProvider +{ + @Override + SinkProto.TransactionMetadata convertToTargetRecord(T record) + { + ByteBuffer buffer = (ByteBuffer) record; + + try + { + SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(buffer); + metricsFacade.recordSerdTxChange(); + return tx; + } catch (InvalidProtocolBufferException e) + { + throw new RuntimeException(e); + } + + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java index 85dd1a4..43681f6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java @@ -17,7 +17,6 @@ package io.pixelsdb.pixels.sink.sink; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkDefaultConfig; @@ -102,7 +101,7 @@ public void flush() } @Override - public boolean write(RowChangeEvent event) + public boolean writeRow(RowChangeEvent event) { final String tableName = event.getTable(); if (event.getOp() == SinkProto.OperationType.DELETE) @@ -139,15 +138,10 @@ public boolean write(RowChangeEvent event) } @Override - public boolean writeTrans(String schemaName, List tableUpdateData, long timestamp) + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { - throw new UnsupportedOperationException("CSV Writer doesn't implement batch write trans"); - } - - @Override - public boolean writeBatch(String schemaName, List tableUpdateData) - { - throw new UnsupportedOperationException("CSV Writer doesn't implement batch write batch"); + // TODO(AntiO2): Write Trans info + return false; } private FileChannel getOrCreateChannel(RowChangeEvent event) throws IOException diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java b/src/main/java/io/pixelsdb/pixels/sink/sink/NoneWriter.java similarity index 53% rename from src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/NoneWriter.java index daff13a..d4551c5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/NoneWriter.java @@ -15,23 +15,36 @@ * */ -package io.pixelsdb.pixels.sink.concurrent; +package io.pixelsdb.pixels.sink.sink; -public class TransactionCoordinatorFactory +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; + +import java.io.IOException; + +public class NoneWriter implements PixelsSinkWriter { - private static TransactionCoordinator instance; + @Override + public void flush() + { + + } - public static synchronized TransactionCoordinator getCoordinator() + @Override + public boolean writeRow(RowChangeEvent rowChangeEvent) { - if (instance == null) - { - instance = new TransactionCoordinator(); - } - return instance; + return true; } - public static synchronized void reset() + @Override + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { - instance = null; + return true; + } + + @Override + public void close() throws IOException + { + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java index d04acb5..f836115 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java @@ -17,19 +17,17 @@ package io.pixelsdb.pixels.sink.sink; -import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import java.io.Closeable; -import java.util.List; public interface PixelsSinkWriter extends Closeable { void flush(); - boolean write(RowChangeEvent rowChangeEvent); + boolean writeRow(RowChangeEvent rowChangeEvent); - boolean writeTrans(String schemaName, List tableUpdateData, long timestamp); + boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata); - boolean writeBatch(String schemaName, List tableUpdateData); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java index 4721b07..2423be8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java @@ -19,6 +19,7 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.sink.retina.RetinaWriter; import java.io.IOException; @@ -26,15 +27,37 @@ public class PixelsSinkWriterFactory { private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - static public PixelsSinkWriter getWriter() throws IOException + private static volatile PixelsSinkWriter writer = null; + + + static public PixelsSinkWriter getWriter() { - switch (config.getPixelsSinkMode()) + if (writer == null) { - case CSV: - return new CsvWriter(); - case RETINA: - return new RetinaWriter(); + synchronized (PixelsSinkWriterFactory.class) + { + if (writer == null) + { + try + { + switch (config.getPixelsSinkMode()) + { + case CSV: + writer = new CsvWriter(); + case RETINA: + writer = new RetinaWriter(); + case PROTO: + writer = new ProtoWriter(); + case NONE: + writer = new NoneWriter(); + } + } catch (IOException e) + { + throw new RuntimeException("Can't create writer", e); + } + } + } } - return null; + return writer; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java index 93044a9..0dbfd5b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java @@ -20,11 +20,9 @@ import io.pixelsdb.pixels.common.physical.PhysicalWriter; -import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.event.ProtoType; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; @@ -34,7 +32,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.List; /** * @package: io.pixelsdb.pixels.sink.sink @@ -53,16 +50,17 @@ public ProtoWriter() throws IOException PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); String dataPath = sinkConfig.getSinkProtoData(); - this.writerManager = new RotatingWriterManager(dataPath); + this.writerManager = new RotatingWriterManager(dataPath); this.instance = TableMetadataRegistry.Instance(); } + @Override public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { byte[] transData = transactionMetadata.toByteArray(); return writeData(-1, transData); - // ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); + // ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); // buffer.putInt(ProtoType.TRANS.toInt()); // return writeData(buffer.array(), transData); } @@ -83,13 +81,13 @@ public boolean write(SinkProto.RowRecord rowRecord) return false; } { - return writeData((int)tableId, rowData); + return writeData((int) tableId, rowData); } // ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES + Long.BYTES); // keyBuffer.putInt(ProtoType.ROW.toInt()) // .putLong(tableId); - + // byte[] schemaNameBytes = schemaName.getBytes(); // byte[] tableNameBytes = tableName.getBytes(); @@ -99,6 +97,7 @@ public boolean write(SinkProto.RowRecord rowRecord) // keyBuffer.put(schemaNameBytes).put(tableNameBytes); // return writeData(keyBuffer.array(), rowData); } + // key: -1 means transaction, else means table id private boolean writeData(int key, byte[] data) { @@ -115,10 +114,11 @@ private boolean writeData(byte[] key, byte[] data) return writeBuffer(buf); } - private boolean writeBuffer(ByteBuffer buf) + private synchronized boolean writeBuffer(ByteBuffer buf) { PhysicalWriter writer; - try { + try + { writer = writerManager.current(); writer.prepare(buf.remaining()); writer.append(buf.array()); @@ -131,13 +131,11 @@ private boolean writeBuffer(ByteBuffer buf) } @Override - public boolean write(RowChangeEvent rowChangeEvent) + public boolean writeRow(RowChangeEvent rowChangeEvent) { return write(rowChangeEvent.getRowRecord()); } - - @Override public void flush() { @@ -150,16 +148,4 @@ public void close() throws IOException { this.writerManager.close(); } - - @Override - public boolean writeTrans(String schemaName, List tableUpdateData, long timestamp) - { - return false; - } - - @Override - public boolean writeBatch(String schemaName, List tableUpdateData) - { - return false; - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java b/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java index 66e8c8f..7705d09 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java @@ -39,13 +39,12 @@ public class RotatingWriterManager private final String baseDir; private final String topic; private final int maxRecordsPerFile; - + private final Storage.Scheme scheme; + private final EtcdFileRegistry registry; private int currentCount = 0; private PhysicalWriter currentWriter; private String currentFileName; - private final Storage.Scheme scheme; - private final EtcdFileRegistry registry; public RotatingWriterManager(String topic) throws IOException { PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaServiceProxy.java similarity index 51% rename from src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaServiceProxy.java index 7a678b3..5f01229 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaServiceProxy.java @@ -15,16 +15,15 @@ * */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.sink.retina; import io.pixelsdb.pixels.common.exception.RetinaException; import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.retina.RetinaProto; -import io.pixelsdb.pixels.sink.concurrent.TransactionMode; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,9 +33,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; -public class RetinaWriter implements PixelsSinkWriter +public class RetinaServiceProxy { - private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriter.class); + private static final Logger LOGGER = LoggerFactory.getLogger(RetinaServiceProxy.class); @Getter private static final PixelsSinkMode pixelsSinkMode = PixelsSinkMode.RETINA; private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); @@ -45,7 +44,8 @@ public class RetinaWriter implements PixelsSinkWriter private final RetinaService retinaService = RetinaService.Instance(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private RetinaService.StreamHandler retinaStream = null; - public RetinaWriter() + + public RetinaServiceProxy() { if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { @@ -56,66 +56,7 @@ public RetinaWriter() } } - @Override - public void flush() - { - } - - @Override - public boolean write(RowChangeEvent event) - { - if (isClosed.get()) - { - LOGGER.warn("Attempted to write to closed writer"); - return false; - } - - try - { - switch (event.getOp()) - { - case INSERT: - case SNAPSHOT: - return sendInsertRequest(event); - case UPDATE: - return sendUpdateRequest(event); - case DELETE: - return sendDeleteRequest(event); - case UNRECOGNIZED: - break; - } - } catch (RetinaException e) - { - LOGGER.error("Retina write failed for event {}", event); - return false; - } - // TODO: error handle - return false; - } - - @Override - public boolean writeTrans(String schemaName, List tableUpdateData, long timestamp) - { - if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) - { - try - { - LOGGER.debug("Retina Writer update record {}, TS: {}", schemaName, timestamp); - retinaService.updateRecord(schemaName, tableUpdateData); - } catch (RetinaException e) - { - e.printStackTrace(); - return false; - } - } else - { - retinaStream.updateRecord(schemaName, tableUpdateData); - } - return true; - } - - @Override - public boolean writeBatch(String schemaName, List tableUpdateData) + public boolean writeTrans(String schemaName, List tableUpdateData) { if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) { @@ -135,7 +76,7 @@ public boolean writeBatch(String schemaName, List t } public CompletableFuture writeBatchAsync - (String schemaName, List tableUpdateData) + (String schemaName, List tableUpdateData) { if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) { @@ -153,58 +94,15 @@ public boolean writeBatch(String schemaName, List t } } - - @Deprecated - private boolean sendInsertRequest(RowChangeEvent event) throws RetinaException - { - // Insert retina - // boolean retinaServiceResult = retinaService.insertRecord(event.getSchemaName(), event.getTable(), event.getAfterData(), event.getTimeStamp()); - - return false; - } - - private boolean sendDeleteRequest(RowChangeEvent event) - { - return false; -// RetinaProto.DeleteRecordResponse deleteRecordResponse = blockingStub.deleteRecord(getDeleteRecordRequest(event)); -// return deleteRecordResponse.getHeader().getErrorCode() == 0; - } - - @Override public void close() throws IOException { - if (isClosed.compareAndSet(false, true)) - { -// try { -// channel.shutdown(); -// if (!channel.awaitTermination(5, TimeUnit.SECONDS)) { -// channel.shutdownNow(); -// } -// } catch (InterruptedException e) { -// Thread.currentThread().interrupt(); -// throw new IOException("Channel shutdown interrupted", e); -// } - } - + isClosed.compareAndSet(false, true); if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { retinaStream.close(); } } - private boolean sendUpdateRequest(RowChangeEvent event) - { - // Delete & Insert -// RetinaProto.DeleteRecordResponse deleteRecordResponse = blockingStub.deleteRecord(getDeleteRecordRequest(event)); -// if (deleteRecordResponse.getHeader().getErrorCode() != 0) { -// return false; -// } -// -// RetinaProto.InsertRecordResponse insertRecordResponse = blockingStub.insertRecord(getInsertRecordRequest(event)); -// return insertRecordResponse.getHeader().getErrorCode() == 0; - return false; - } - public enum RetinaWriteMode { STREAM, diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java new file mode 100644 index 0000000..9159363 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java @@ -0,0 +1,300 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.sink.retina; + +import io.pixelsdb.pixels.common.exception.TransException; +import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; +import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.prometheus.client.Summary; +import org.apache.commons.lang3.RandomUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +public class RetinaWriter implements PixelsSinkWriter +{ + private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriter.class); + final ExecutorService dispatchExecutor = Executors.newCachedThreadPool(); + private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(1024); + private final ScheduledExecutorService timeoutScheduler = + Executors.newSingleThreadScheduledExecutor(); + private final TransactionProxy transactionProxy = TransactionProxy.Instance(); + private final TransService transService; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + + private final SinkContextManager sinkContextManager; + + public RetinaWriter() + { + transService = TransService.Instance(); + this.sinkContextManager = SinkContextManager.getInstance(); + } + + @Override + public boolean writeTrans(SinkProto.TransactionMetadata txMeta) + { + try + { + if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) + { + handleTxBegin(txMeta); + } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) + { + handleTxEnd(txMeta); + } + } catch (SinkException e) + { + LOGGER.error(e.getMessage(), e); + return false; + } + return true; + } + + @Override + public boolean writeRow(RowChangeEvent event) + { + try + { + if (event == null) + { + return false; + } + + metricsFacade.recordRowChange(event.getTable(), event.getOp()); + event.startLatencyTimer(); + if (event.getTransaction() == null || event.getTransaction().getId().isEmpty()) + { + handleNonTxEvent(event); + return true; + } + + + String table = event.getFullTableName(); + + long collectionOrder = event.getTransaction().getDataCollectionOrder(); + long totalOrder = event.getTransaction().getTotalOrder(); + + AtomicBoolean canWrite = new AtomicBoolean(false); + SinkContext ctx = sinkContextManager.getActiveTxContext(event, canWrite); + + if (canWrite.get()) + { + sinkContextManager.writeRowChangeEvent(ctx, event); + } + } catch (SinkException e) + { + LOGGER.error(e.getMessage(), e); + return false; + } + + return true; + } + + private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkException + { + // startTrans(txBegin.getId()).get(); + try + { + startTransSync(txBegin.getId()); + } catch (SinkException e) + { + throw new SinkException("Failed to start trans", e); + } + + } + + private void startTransSync(String sourceTxId) throws SinkException + { + sinkContextManager.startTransSync(sourceTxId); + } + + private void handleTxEnd(SinkProto.TransactionMetadata txEnd) + { + String txId = txEnd.getId(); + SinkContext ctx = sinkContextManager.getSinkContext(txId); + + transactionExecutor.submit(() -> + { + processTxCommit(txEnd, txId, ctx); + } + ); + switch (pixelsSinkConfig.getTransactionMode()) + { + +// case BATCH -> +// { +// processTxCommit(txEnd, txId, ctx); +// } +// case RECORD -> +// { +// transactionExecutor.submit(() -> +// { +// processTxCommit(txEnd, txId, ctx); +// } +// ); +// } + } + } + + private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, SinkContext ctx) + { + LOGGER.trace("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), + txEnd.getDataCollectionsList().stream() + .map(dc -> dc.getDataCollection() + "=" + + ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + + "/" + dc.getEventCount()) + .collect(Collectors.joining(", "))); + if (ctx == null) + { + LOGGER.warn("Sink Context is null"); + return; + } + + try + { + try + { + ctx.tableCounterLock.lock(); + while (!ctx.isCompleted(txEnd)) + { + LOGGER.debug("TX End Get Lock {}", txId); + LOGGER.debug("Waiting for events in TX {}: {}", txId, + txEnd.getDataCollectionsList().stream() + .map(dc -> dc.getDataCollection() + "=" + + ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + + "/" + dc.getEventCount()) + .collect(Collectors.joining(", "))); + ctx.tableCounterCond.await(); + } + } finally + { + ctx.tableCounterLock.unlock(); + } + + + sinkContextManager.removeSinkContext(txId); + boolean res = true; + if (res) + { + LOGGER.trace("Committed transaction: {}", txId); + Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); + transactionProxy.commitTransAsync(ctx.getPixelsTransCtx()); + } else + { + LOGGER.info("Abort transaction: {}", txId); + Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); + CompletableFuture.runAsync(() -> + { + try + { + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); + } catch (TransException e) + { + throw new RuntimeException(e); + } + }).whenComplete((v, ex) -> + { + transLatencyTimer.close(); + if (ex != null) + { + LOGGER.error("Rollback failed", ex); + } + }); + } + } catch (InterruptedException e) + { + try + { + LOGGER.info("Catch Exception, Abort transaction: {}", txId); + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); + } catch (TransException ex) + { + LOGGER.error("Failed to abort transaction {}", txId); + ex.printStackTrace(); + LOGGER.error(ex.getMessage()); + throw new RuntimeException(ex); + } + LOGGER.error(e.getMessage()); + LOGGER.error("Failed to commit transaction {}", txId, e); + } + } + + + private void handleNonTxEvent(RowChangeEvent event) throws SinkException + { + // virtual tx + String randomId = Long.toString(System.currentTimeMillis()) + RandomUtils.nextLong(); + writeTrans(buildBeginTransactionMetadata(randomId)); + sinkContextManager.writeRowChangeEvent(randomId, event); + writeTrans(buildEndTransactionMetadata(event.getFullTableName(), randomId)); + } + + public void shutdown() + { + dispatchExecutor.shutdown(); + timeoutScheduler.shutdown(); + } + + @Override + public void close() throws IOException + { + + } + + @Override + public void flush() + { + + } + + private SinkProto.TransactionMetadata buildBeginTransactionMetadata(String id) + { + SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); + builder.setStatus(SinkProto.TransactionStatus.BEGIN) + .setId(id); + return builder.build(); + } + + private SinkProto.TransactionMetadata buildEndTransactionMetadata(String fullTableName, String id) + { + SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); + builder.setStatus(SinkProto.TransactionStatus.END) + .setId(id) + .setEventCount(1L); + + SinkProto.DataCollection.Builder dataCollectionBuilder = SinkProto.DataCollection.newBuilder(); + dataCollectionBuilder.setDataCollection(fullTableName) + .setEventCount(1L); + builder.addDataCollections(dataCollectionBuilder); + return builder.build(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContext.java similarity index 87% rename from src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContext.java index ddd84e3..61d310a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContext.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.concurrent; +package io.pixelsdb.pixels.sink.sink.retina; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.sink.SinkProto; @@ -30,7 +30,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -56,13 +55,13 @@ public class SinkContext Queue orphanEvent = new ConcurrentLinkedQueue<>(); private TransContext pixelsTransCtx; - SinkContext(String sourceTxId) + public SinkContext(String sourceTxId) { this.sourceTxId = sourceTxId; this.pixelsTransCtx = null; } - SinkContext(String sourceTxId, TransContext pixelsTransCtx) + public SinkContext(String sourceTxId, TransContext pixelsTransCtx) { this.sourceTxId = sourceTxId; this.pixelsTransCtx = pixelsTransCtx; @@ -128,26 +127,9 @@ boolean isCompleted(SinkProto.TransactionMetadata tx) return true; } - boolean isExpired() - { - // TODO: expire timeout transaction - return false; - // return System.currentTimeMillis() - pixelsTransCtx.getTimestamp() > TX_TIMEOUT_MS; - } - - void markCompleted() - { - this.completed = true; - } - - void awaitCompletion() throws InterruptedException, ExecutionException - { - completionFuture.get(); - } - public long getTimestamp() { - if(pixelsTransCtx == null) + if (pixelsTransCtx == null) { throw new RuntimeException("PixelsTransCtx is NULL"); } @@ -214,13 +196,4 @@ public void setPixelsTransCtx(TransContext pixelsTransCtx) this.pixelsTransCtx = pixelsTransCtx; } - public boolean isCompleted() - { - return completed; - } - - public void setCompleted(boolean completed) - { - this.completed = completed; - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java new file mode 100644 index 0000000..2f4e3ce --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java @@ -0,0 +1,139 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.sink.retina; + +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; + +public class SinkContextManager +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SinkContextManager.class); + private final static SinkContextManager INSTANCE = new SinkContextManager(); + + private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(); + + private final TransactionProxy transactionProxy = TransactionProxy.Instance(); + private final TableWriterProxy tableWriterProxy; + + private SinkContextManager() + { + this.tableWriterProxy = TableWriterProxy.getInstance(); + } + + public static SinkContextManager getInstance() + { + return INSTANCE; + } + + protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean canWrite) + { + String txId = event.getTransaction().getId(); + return activeTxContexts.compute(txId, (sourceTxId, sinkContext) -> + { + if (sinkContext == null) + { + SinkContext newSinkContext = new SinkContext(sourceTxId); + newSinkContext.bufferOrphanedEvent(event); + return newSinkContext; + } else + { + if (sinkContext.getPixelsTransCtx() == null) + { + sinkContext.bufferOrphanedEvent(event); + return sinkContext; + } + canWrite.set(true); + return sinkContext; + } + }); + } + + protected void startTransSync(String sourceTxId) + { + activeTxContexts.compute( + sourceTxId, + (k, oldCtx) -> + { + if (oldCtx == null) + { + return new SinkContext(sourceTxId, transactionProxy.getNewTransContext()); + } else + { + oldCtx.getLock().lock(); + try + { + oldCtx.setPixelsTransCtx(transactionProxy.getNewTransContext()); + handleOrphanEvents(oldCtx); + oldCtx.getCond().signalAll(); + } catch (SinkException e) + { + throw new RuntimeException(e); + } finally + { + oldCtx.getLock().unlock(); + } + return oldCtx; + } + } + ); + LOGGER.trace("Begin Tx Sync: {}", sourceTxId); + } + + private void handleOrphanEvents(SinkContext ctx) throws SinkException + { + Queue buffered = ctx.getOrphanEvent(); + + if (buffered != null) + { + for (RowChangeEvent event : buffered) + { + writeRowChangeEvent(ctx, event); + } + } + } + + protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException + { + String table = event.getTable(); + event.setTimeStamp(ctx.getTimestamp()); + event.initIndexKey(); + tableWriterProxy.getTableWriter(table).write(event, ctx); + } + + protected SinkContext getSinkContext(String txId) + { + return activeTxContexts.get(txId); + } + + protected void removeSinkContext(String txId) + { + activeTxContexts.remove(txId); + } + + protected void writeRowChangeEvent(String randomId, RowChangeEvent event) throws SinkException + { + writeRowChangeEvent(getSinkContext(randomId), event); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableCrossTxWriter.java similarity index 60% rename from src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableCrossTxWriter.java index 728604a..aad0be2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableCrossTxWriter.java @@ -16,22 +16,16 @@ */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.sink.retina; import io.pixelsdb.pixels.retina.RetinaProto; -import io.pixelsdb.pixels.sink.concurrent.SinkContext; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -50,8 +44,8 @@ public class TableCrossTxWriter extends TableWriter private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); private final int flushBatchSize; private final ReentrantLock writeLock = new ReentrantLock(); - private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - public TableCrossTxWriter(String t) throws IOException + + public TableCrossTxWriter(String t) { super(t); flushBatchSize = config.getFlushBatchSize(); @@ -81,7 +75,6 @@ public void flush() writeLock.lock(); try { - String txId = null; String schemaName = null; List smallBatch = null; @@ -89,12 +82,12 @@ public void flush() List fullTableName = new ArrayList<>(); List tableUpdateData = new LinkedList<>(); List tableUpdateCount = new ArrayList<>(); - for(RowChangeEvent event : batch) + for (RowChangeEvent event : batch) { String currTxId = event.getTransaction().getId(); - if(!currTxId.equals(txId)) + if (!currTxId.equals(txId)) { - if(smallBatch != null && !smallBatch.isEmpty()) + if (smallBatch != null && !smallBatch.isEmpty()) { tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch)); tableUpdateCount.add(smallBatch.size()); @@ -107,23 +100,17 @@ public void flush() smallBatch.add(event); } - if(smallBatch != null) + if (smallBatch != null) { tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch)); tableUpdateCount.add(smallBatch.size()); } - if(sinkMode == PixelsSinkMode.NONE) - { - // for test reason - updateCtxCounters(txIds, fullTableName, tableUpdateCount); - return; - } - CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); updateRecordResponseCompletableFuture.thenAccept( - resp -> { + resp -> + { updateCtxCounters(txIds, fullTableName, tableUpdateCount); } ); @@ -135,49 +122,49 @@ public void flush() private void updateCtxCounters(List txIds, List fullTableName, List tableUpdateCount) { - for(int i = 0; i < txIds.size(); i++) + for (int i = 0; i < txIds.size(); i++) { metricsFacade.recordRowEvent(tableUpdateCount.get(i)); String writeTxId = txIds.get(i); - SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(writeTxId); + SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); } } private RetinaProto.TableUpdateData buildTableUpdateDataFromBatch(String txId, List smallBatch) { - SinkContext sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(txId); - try - { - sinkContext.getLock().lock(); - while (sinkContext.getPixelsTransCtx() == null) - { - LOGGER.warn("Wait for tx to begin trans: {}", txId); // CODE SHOULD NOT REACH HERE - sinkContext.getCond().await(); - } - } catch (InterruptedException e) - { - throw new RuntimeException(e); - } finally + SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(txId); + try + { + sinkContext.getLock().lock(); + while (sinkContext.getPixelsTransCtx() == null) { - sinkContext.getLock().unlock(); + LOGGER.warn("Wait for tx to begin trans: {}", txId); // CODE SHOULD NOT REACH HERE + sinkContext.getCond().await(); } - RowChangeEvent event1 = smallBatch.get(0); + } catch (InterruptedException e) + { + throw new RuntimeException(e); + } finally + { + sinkContext.getLock().unlock(); + } + RowChangeEvent event1 = smallBatch.get(0); - RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder() - .setTimestamp(sinkContext.getTimestamp()) - .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) - .setTableName(tableName); - try - { - for (RowChangeEvent smallEvent : smallBatch) - { - addUpdateData(smallEvent, builder); - } - } catch (SinkException e) + RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder() + .setTimestamp(sinkContext.getTimestamp()) + .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) + .setTableName(tableName); + try + { + for (RowChangeEvent smallEvent : smallBatch) { - throw new RuntimeException("Flush failed for table " + tableName, e); + addUpdateData(smallEvent, builder); } + } catch (SinkException e) + { + throw new RuntimeException("Flush failed for table " + tableName, e); + } return builder.build(); } @@ -186,18 +173,4 @@ protected boolean needFlush() { return buffer.size() >= flushBatchSize; } - - public static TableWriter getTableWriter(String tableName) - { - return WRITER_REGISTRY.computeIfAbsent(tableName, t -> - { - try - { - return new TableCrossTxWriter(t); - } catch (IOException e) - { - throw new RuntimeException(e); - } - }); - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableSingleTxWriter.java similarity index 78% rename from src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableSingleTxWriter.java index 47fcf2b..3516871 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableSingleTxWriter.java @@ -14,47 +14,27 @@ * limitations under the License. */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.sink.retina; import io.pixelsdb.pixels.retina.RetinaProto; -import io.pixelsdb.pixels.sink.concurrent.SinkContext; -import io.pixelsdb.pixels.sink.concurrent.TransactionCoordinatorFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; -import java.util.concurrent.*; -import java.util.concurrent.locks.ReentrantLock; public class TableSingleTxWriter extends TableWriter { + private static final long TX_TIMEOUT_MS = 3000; @Getter private final Logger LOGGER = LoggerFactory.getLogger(TableSingleTxWriter.class); - private static final long TX_TIMEOUT_MS = 3000; - public TableSingleTxWriter(String tableName) throws IOException - { - super(tableName); - } - public static TableWriter getTableWriter(String tableName) + public TableSingleTxWriter(String tableName) { - return WRITER_REGISTRY.computeIfAbsent(tableName, t -> - { - try - { - return new TableSingleTxWriter(t); - } catch (IOException e) - { - throw new RuntimeException(e); - } - }); + super(tableName); } /** @@ -76,7 +56,7 @@ public void flush() txId = currentTxId; currentTxId = null; - sinkContext = TransactionCoordinatorFactory.getCoordinator().getSinkContext(txId); + sinkContext = SinkContextManager.getInstance().getSinkContext(txId); sinkContext.getLock().lock(); try { @@ -115,7 +95,7 @@ public void flush() addUpdateData(event, builder); } List tableUpdateData = List.of(builder.build()); - delegate.writeTrans(event1.getSchemaName(), tableUpdateData, sinkContext.getTimestamp()); + delegate.writeTrans(event1.getSchemaName(), tableUpdateData); sinkContext.updateCounter(fullTableName, batch.size()); // ---- Outside lock: build proto and write ---- LOGGER.info("Flushing {} events for table {} txId={}", batch.size(), fullTableName, txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriter.java similarity index 84% rename from src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriter.java index cec4056..3da1a47 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriter.java @@ -16,24 +16,24 @@ */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.sink.retina; import io.pixelsdb.pixels.retina.RetinaProto; -import io.pixelsdb.pixels.sink.concurrent.SinkContext; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import java.io.IOException; -import java.util.ArrayList; import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; /** @@ -45,11 +45,11 @@ public abstract class TableWriter { - // TODO(AntiO2): 这里放弃掉底层writer的多种实现了。 - protected final RetinaWriter delegate; // physical writer + protected final RetinaServiceProxy delegate; // physical writer protected final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); protected final ReentrantLock lock = new ReentrantLock(); protected final String tableName; + protected final long flushInterval; // Shared state (protected by lock) protected List buffer = new LinkedList<>(); protected volatile String currentTxId = null; @@ -57,22 +57,41 @@ public abstract class TableWriter protected ScheduledFuture flushTask = null; protected String fullTableName; protected PixelsSinkConfig config; - protected PixelsSinkMode sinkMode; - private MetricsFacade metricsFacade = MetricsFacade.getInstance(); + protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); + - protected TableWriter(String tableName) throws IOException + protected TableWriter(String tableName) { this.config = PixelsSinkConfigFactory.getInstance(); this.tableName = tableName; - this.delegate = new RetinaWriter(); + this.delegate = new RetinaServiceProxy(); this.flushInterval = config.getFlushIntervalMs(); - this.sinkMode = config.getPixelsSinkMode(); + } - - protected static final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); - protected abstract Logger getLOGGER(); - protected final long flushInterval; + /** + * Helper: add insert/delete data into proto builder. + */ + protected static void addUpdateData(RowChangeEvent rowChangeEvent, + RetinaProto.TableUpdateData.Builder builder) throws SinkException + { + if (rowChangeEvent.hasBeforeData()) + { + RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); + deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); + builder.addDeleteData(deleteDataBuilder); + } + + if (rowChangeEvent.hasAfterData()) + { + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); + insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); + insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); + builder.addInsertData(insertDataBuilder); + } + } + + protected abstract Logger getLOGGER(); public boolean write(RowChangeEvent event, SinkContext ctx) { @@ -83,7 +102,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) { txId = ctx.getSourceTxId(); // If this is a new transaction, flush the old one - if(needFlush()) + if (needFlush()) { if (flushTask != null) { @@ -154,25 +173,4 @@ public void close() throw new RuntimeException(e); } } - /** - * Helper: add insert/delete data into proto builder. - */ - public static void addUpdateData(RowChangeEvent rowChangeEvent, - RetinaProto.TableUpdateData.Builder builder) throws SinkException - { - if (rowChangeEvent.hasBeforeData()) - { - RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); - deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); - builder.addDeleteData(deleteDataBuilder); - } - - if (rowChangeEvent.hasAfterData()) - { - RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); - insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); - insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); - builder.addInsertData(insertDataBuilder); - } - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java new file mode 100644 index 0000000..a347d7a --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java @@ -0,0 +1,65 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.sink.retina; + +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class TableWriterProxy +{ + private final static TableWriterProxy INSTANCE = new TableWriterProxy(); + + private final TransactionMode transactionMode; + private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); + + private TableWriterProxy() + { + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + this.transactionMode = pixelsSinkConfig.getTransactionMode(); + } + + protected static TableWriterProxy getInstance() + { + return INSTANCE; + } + + protected TableWriter getTableWriter(String tableName) + { + return WRITER_REGISTRY.computeIfAbsent(tableName, t -> + { + switch (transactionMode) + { + case SINGLE -> + { + return new TableSingleTxWriter(t); + } + case BATCH -> + { + return new TableCrossTxWriter(t); + } + default -> + { + throw new IllegalArgumentException("Unknown transaction mode: " + transactionMode); + } + } + }); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionMode.java similarity index 92% rename from src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionMode.java index ff97bb8..788b0ad 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionMode.java @@ -15,13 +15,12 @@ * */ -package io.pixelsdb.pixels.sink.concurrent; +package io.pixelsdb.pixels.sink.sink.retina; public enum TransactionMode { - TRANS, - BATCH, - RECORD; + SINGLE, + BATCH; public static TransactionMode fromValue(String value) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionProxy.java similarity index 90% rename from src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java rename to src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionProxy.java index d94f869..b11b60f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/concurrent/TransactionManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionProxy.java @@ -15,15 +15,14 @@ * */ -package io.pixelsdb.pixels.sink.concurrent; +package io.pixelsdb.pixels.sink.sink.retina; -import io.debezium.pipeline.txmetadata.TransactionContext; import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,14 +32,14 @@ import java.util.concurrent.*; /** - * This class if for + * This class if for pixels trans service * * @author AntiO2 */ -public class TransactionManager +public class TransactionProxy { - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionManager.class); - private final static TransactionManager instance = new TransactionManager(); + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionProxy.class); + private final static TransactionProxy instance = new TransactionProxy(); private final TransService transService; private final Queue transContextQueue; private final Object batchLock = new Object(); @@ -53,7 +52,7 @@ public class TransactionManager private final int MAX_WAIT_MS; - TransactionManager() + private TransactionProxy() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); BATCH_SIZE = pixelsSinkConfig.getCommitBatchSize(); @@ -65,7 +64,8 @@ public class TransactionManager this.toCommitTransContextQueue = new LinkedBlockingQueue<>(); this.batchCommitExecutor = Executors.newFixedThreadPool( WORKER_COUNT, - r -> { + r -> + { Thread t = new Thread(r); t.setName("commit-trans-batch-thread"); t.setDaemon(true); @@ -80,7 +80,7 @@ public class TransactionManager } - public static TransactionManager Instance() + public static TransactionProxy Instance() { return instance; } @@ -97,7 +97,7 @@ private void requestTransactions() } } - public TransContext getTransContext() + public TransContext getNewTransContext() { TransContext ctx = transContextQueue.poll(); if (ctx != null) @@ -171,18 +171,15 @@ private void batchCommitWorker() batchTransIds.size(), (System.nanoTime() - startTime) / 1_000_000); } - } - catch (InterruptedException ie) + } catch (InterruptedException ie) { LOGGER.warn("Batch commit worker interrupted, exiting..."); Thread.currentThread().interrupt(); break; - } - catch (TransException e) + } catch (TransException e) { LOGGER.error("Batch commit failed: {}", e.getMessage(), e); - } - catch (Exception e) + } catch (Exception e) { LOGGER.error("Unexpected error in batch commit worker", e); } @@ -193,7 +190,7 @@ public void close() { synchronized (batchLock) { - while(true) + while (true) { TransContext ctx = transContextQueue.poll(); if (ctx == null) @@ -202,7 +199,7 @@ public void close() } try { - transService.rollbackTrans(ctx.getTransId(),false); + transService.rollbackTrans(ctx.getTransId(), false); } catch (TransException e) { throw new RuntimeException(e); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java similarity index 56% rename from src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java rename to src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 39167d7..3d964e1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/FasterSinkStorageProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -1,107 +1,99 @@ -/* - * Copyright 2025 PixelsDB. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - - -package io.pixelsdb.pixels.sink.processor; - - -import io.pixelsdb.pixels.common.metadata.SchemaTableName; +package io.pixelsdb.pixels.sink.source; + import io.pixelsdb.pixels.common.physical.PhysicalReader; import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; import io.pixelsdb.pixels.common.physical.Storage; import io.pixelsdb.pixels.core.utils.Pair; -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.event.*; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import io.pixelsdb.pixels.sink.provider.*; +import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.List; import java.util.Map; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; -/** - * @package: io.pixelsdb.pixels.sink.processor - * @className: SinkStorageProcessor - * @author: AntiO2 - * @date: 2025/10/5 11:43 - */ -public class FasterSinkStorageProcessor extends AbstractSinkStorageProcessor implements MainProcessor +public abstract class AbstractSinkStorageSource implements SinkSource { - private static final Logger LOGGER = LoggerFactory.getLogger(FasterSinkStorageProcessor.class); + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractSinkStorageSource.class); + protected final AtomicBoolean running = new AtomicBoolean(false); - private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; - private final TableStoragePipelineManager tablePipelineManager = new TableStoragePipelineManager(); - private final TransactionProcessor transactionProcessor = new TransactionProcessor(transactionEventProvider); - private final Thread transactionProcessorThread; - private final Thread transAdapterThread; - private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final Map>> queueMap = new ConcurrentHashMap<>(); - private final Map consumerThreads = new ConcurrentHashMap<>(); - private final int maxQueueCapacity = 10000; + protected final String topic; + protected final String baseDir; + protected final EtcdFileRegistry etcdFileRegistry; + protected final List files; + + protected TransactionEventStorageProvider transactionEventProvider; + protected TransactionProcessor transactionProcessor; + protected Thread transactionProviderThread; + protected Thread transactionProcessorThread; + private final Map consumerThreads = new ConcurrentHashMap<>(); + private final int maxQueueCapacity = 10000; private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - public FasterSinkStorageProcessor() + protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); + private final Map>> queueMap = new ConcurrentHashMap<>(); + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final TableProviderAndProcessorPipelineManager tablePipelineManager = new TableProviderAndProcessorPipelineManager(); + + protected AbstractSinkStorageSource() { - this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); - this.transAdapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + this.topic = pixelsSinkConfig.getSinkProtoData(); + this.baseDir = pixelsSinkConfig.getSinkProtoDir(); + this.etcdFileRegistry = new EtcdFileRegistry(topic, baseDir); + this.files = this.etcdFileRegistry.listAllFiles(); + this.transactionEventProvider = new TransactionEventStorageProvider<>(); + this.transactionProviderThread = new Thread(transactionEventProvider); + + this.transactionProcessor = new TransactionProcessor(transactionEventProvider); + this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); } - @Override - ProtoType getProtoType(int i) + abstract ProtoType getProtoType(int i); + + protected void handleTransactionSourceRecord(ByteBuffer record) { - if(i == -1) - { - return ProtoType.TRANS; - } - return ProtoType.ROW; + transactionEventProvider.putTransRawEvent(record); } - private final CompletableFuture POISON_PILL = new CompletableFuture<>(); - @Override public void start() { this.transactionProcessorThread.start(); - this.transAdapterThread.start(); - ExecutorService transExecutor = Executors.newSingleThreadExecutor(); - ExecutorService rowExecutor = Executors.newSingleThreadExecutor(); - for(String file:files) + this.transactionProviderThread.start(); + for (String file : files) { Storage.Scheme scheme = Storage.Scheme.fromPath(file); LOGGER.info("Start read from file {}", file); - try(PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) + try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) { long offset = 0; BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); BlockingQueue> transQueue = new LinkedBlockingQueue<>(); - while(true) + while (true) { - try { + try + { int key, valueLen; reader.seek(offset); - try { + try + { key = reader.readInt(ByteOrder.BIG_ENDIAN); valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) { + } catch (IOException e) + { // EOF break; } @@ -124,13 +116,15 @@ public void start() queue.put(valueFuture); // Start consumer thread if not exists - consumerThreads.computeIfAbsent(key, k -> { + consumerThreads.computeIfAbsent(key, k -> + { Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); t.setName("consumer-" + key); t.start(); return t; }); - } catch (IOException | InterruptedException e) { + } catch (IOException | InterruptedException e) + { break; } } @@ -142,89 +136,72 @@ public void start() } // signal all queues to stop - queueMap.values().forEach(q -> { - try { + queueMap.values().forEach(q -> + { + try + { q.put(POISON_PILL); - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } }); // wait all consumers to finish - consumerThreads.values().forEach(t -> { - try { + consumerThreads.values().forEach(t -> + { + try + { t.join(); - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } }); } - private ByteBuffer copyToHeap(ByteBuffer directBuffer) { - ByteBuffer duplicate = directBuffer.duplicate(); - ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); - heapBuffer.put(duplicate); - heapBuffer.flip(); - return heapBuffer; - } - private static String readString(ByteBuffer buffer, int len) { - byte[] bytes = new byte[len]; - buffer.get(bytes); - return new String(bytes); - } - - private void consumeQueue(int key, BlockingQueue> queue, ProtoType protoType) { - try { - while (true) { + private void consumeQueue(int key, BlockingQueue> queue, ProtoType protoType) + { + try + { + while (true) + { CompletableFuture value = queue.take(); - if(value == POISON_PILL) + if (value == POISON_PILL) { break; } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - switch (protoType) { + switch (protoType) + { case ROW -> handleRowChangeSourceRecord(key, valueBuffer); case TRANS -> handleTransactionSourceRecord(valueBuffer); } } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); - } catch (ExecutionException e) { + } catch (ExecutionException e) + { LOGGER.error("Error in async processing", e); } } - static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); + + private ByteBuffer copyToHeap(ByteBuffer directBuffer) + { + ByteBuffer duplicate = directBuffer.duplicate(); + ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); + heapBuffer.put(duplicate); + heapBuffer.flip(); + return heapBuffer; + } private void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer) { tablePipelineManager.routeRecord(key, dataBuffer); } - private void processTransactionSourceRecord() { - while (true) { - ByteBuffer data = null; - try - { - data = rawTransactionQueue.take(); - } catch (InterruptedException e) - { - throw new RuntimeException(e); - } - try { - SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(data); - metricsFacade.recordSerdTxChange(); - if (tx != null) { - transactionEventProvider.getEventQueue().put(tx); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } @Override public boolean isRunning() @@ -235,7 +212,8 @@ public boolean isRunning() @Override public void stopProcessor() { - transAdapterThread.interrupt(); + transactionProviderThread.interrupt(); + transactionProcessorThread.interrupt(); transactionProcessor.stopProcessor(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java new file mode 100644 index 0000000..2d0e513 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java @@ -0,0 +1,77 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + + +package io.pixelsdb.pixels.sink.source; + + +import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; +import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.core.utils.Pair; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import io.pixelsdb.pixels.sink.provider.ProtoType; +import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; +import io.pixelsdb.pixels.sink.provider.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.provider.TransactionEventStorageProvider; +import io.pixelsdb.pixels.sink.util.MetricsFacade; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +import java.util.concurrent.*; + +/** + * @package: io.pixelsdb.pixels.sink.processor + * @className: LegacySinkStorageSource + * @author: AntiO2 + * @date: 2025/10/5 11:43 + */ +public class FasterSinkStorageSource extends AbstractSinkStorageSource implements SinkSource +{ + private static final Logger LOGGER = LoggerFactory.getLogger(FasterSinkStorageSource.class); + static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); + + public FasterSinkStorageSource() + { + super(); + } + + private static String readString(ByteBuffer buffer, int len) + { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return new String(bytes); + } + + @Override + ProtoType getProtoType(int i) + { + if (i == -1) + { + return ProtoType.TRANS; + } + return ProtoType.ROW; + } + +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java similarity index 69% rename from src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java rename to src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java index f20a98d..b461a31 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkStorageProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.processor; +package io.pixelsdb.pixels.sink.source; import io.pixelsdb.pixels.common.metadata.SchemaTableName; @@ -25,51 +25,49 @@ import io.pixelsdb.pixels.common.physical.Storage; import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.event.ProtoType; -import io.pixelsdb.pixels.sink.event.TableEnginePipelineManager; -import io.pixelsdb.pixels.sink.event.TransactionEventEngineProvider; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; +import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import io.pixelsdb.pixels.sink.provider.ProtoType; +import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; +import io.pixelsdb.pixels.sink.provider.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.List; import java.util.Map; import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicBoolean; /** * @package: io.pixelsdb.pixels.sink.processor - * @className: SinkStorageProcessor + * @className: LegacySinkStorageSource * @author: AntiO2 * @date: 2025/10/5 11:43 */ -public class SinkStorageProcessor extends AbstractSinkStorageProcessor implements MainProcessor +@Deprecated +public class LegacySinkStorageSource extends AbstractSinkStorageSource implements SinkSource { - private static final Logger LOGGER = LoggerFactory.getLogger(SinkStorageProcessor.class); - + private static final Logger LOGGER = LoggerFactory.getLogger(LegacySinkStorageSource.class); + static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; - private final TableEnginePipelineManager tableEnginePipelineManager = new TableEnginePipelineManager(); + + private final TableProviderAndProcessorPipelineManager tableProvidersManagerImpl = new TableProviderAndProcessorPipelineManager<>(); private final TransactionProcessor transactionProcessor = new TransactionProcessor(transactionEventProvider); - private final Thread transactionProcessorThread; - private final Thread transAdapterThread; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final Map>> queueMap = new ConcurrentHashMap<>(); private final Map consumerThreads = new ConcurrentHashMap<>(); private final int maxQueueCapacity = 10000; + private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + private final CompletableFuture POISON_PILL = new CompletableFuture<>(); - private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - public SinkStorageProcessor() + private static String readString(ByteBuffer buffer, int len) { - this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); - this.transAdapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); - + byte[] bytes = new byte[len]; + buffer.get(bytes); + return new String(bytes); } @Override @@ -78,33 +76,31 @@ ProtoType getProtoType(int i) return ProtoType.fromInt(i); } - private final CompletableFuture POISON_PILL = new CompletableFuture<>(); - @Override public void start() { - this.transactionProcessorThread.start(); - this.transAdapterThread.start(); - ExecutorService transExecutor = Executors.newSingleThreadExecutor(); - ExecutorService rowExecutor = Executors.newSingleThreadExecutor(); - for(String file:files) + + for (String file : files) { Storage.Scheme scheme = Storage.Scheme.fromPath(file); LOGGER.info("Start read from file {}", file); - try(PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) + try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) { long offset = 0; BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); BlockingQueue> transQueue = new LinkedBlockingQueue<>(); - while(true) + while (true) { - try { + try + { int keyLen, valueLen; reader.seek(offset); - try { + try + { keyLen = reader.readInt(ByteOrder.BIG_ENDIAN); valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) { + } catch (IOException e) + { // EOF break; } @@ -130,13 +126,15 @@ public void start() queue.put(valueFuture); // Start consumer thread if not exists - consumerThreads.computeIfAbsent(queueKey, k -> { + consumerThreads.computeIfAbsent(queueKey, k -> + { Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); t.setName("consumer-" + queueKey); t.start(); return t; }); - } catch (IOException | InterruptedException e) { + } catch (IOException | InterruptedException e) + { break; } } @@ -148,65 +146,71 @@ public void start() } // signal all queues to stop - queueMap.values().forEach(q -> { - try { + queueMap.values().forEach(q -> + { + try + { q.put(POISON_PILL); - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } }); // wait all consumers to finish - consumerThreads.values().forEach(t -> { - try { + consumerThreads.values().forEach(t -> + { + try + { t.join(); - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } }); } - - - - private ByteBuffer copyToHeap(ByteBuffer directBuffer) { + private ByteBuffer copyToHeap(ByteBuffer directBuffer) + { ByteBuffer duplicate = directBuffer.duplicate(); ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); heapBuffer.put(duplicate); heapBuffer.flip(); return heapBuffer; } - private static String readString(ByteBuffer buffer, int len) { - byte[] bytes = new byte[len]; - buffer.get(bytes); - return new String(bytes); - } - private void consumeQueue(SchemaTableName key, BlockingQueue> queue, ProtoType protoType) { - try { - while (true) { + private void consumeQueue(SchemaTableName key, BlockingQueue> queue, ProtoType protoType) + { + try + { + while (true) + { CompletableFuture value = queue.take(); - if(value == POISON_PILL) + if (value == POISON_PILL) { break; } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - switch (protoType) { + switch (protoType) + { case ROW -> handleRowChangeSourceRecord(key, valueBuffer); case TRANS -> handleTransactionSourceRecord(valueBuffer); } } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); - } catch (ExecutionException e) { + } catch (ExecutionException e) + { LOGGER.error("Error in async processing", e); } } - static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); - private SchemaTableName computeQueueKey(ByteBuffer keyBuffer, ProtoType protoType) { - switch (protoType) { + private SchemaTableName computeQueueKey(ByteBuffer keyBuffer, ProtoType protoType) + { + switch (protoType) + { case ROW -> { int schemaLen = keyBuffer.getInt(); @@ -228,8 +232,9 @@ private SchemaTableName computeQueueKey(ByteBuffer keyBuffer, ProtoType protoTyp private void handleRowChangeSourceRecord(SchemaTableName schemaTableName, ByteBuffer dataBuffer) { - tableEnginePipelineManager.routeRecord(schemaTableName, dataBuffer); + tableProvidersManagerImpl.routeRecord(schemaTableName, dataBuffer); } + private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBuffer) { { @@ -244,44 +249,6 @@ private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBu // } } -// tableEnginePipelineManager.routeRecord(schemaTableName, dataBuffer); - } - - private void processTransactionSourceRecord() { - while (true) { - ByteBuffer data = null; - try - { - data = rawTransactionQueue.take(); - } catch (InterruptedException e) - { - throw new RuntimeException(e); - } - try { - SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(data); - metricsFacade.recordSerdTxChange(); - if (tx != null) { - transactionEventProvider.getEventQueue().put(tx); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } - - @Override - public boolean isRunning() - { - return false; - } - - @Override - public void stopProcessor() - { - transAdapterThread.interrupt(); - transactionProcessor.stopProcessor(); +// tableProvidersManagerImpl.routeRecord(schemaTableName, dataBuffer); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java similarity index 77% rename from src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java rename to src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java index 1f29e26..b8948ce 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java @@ -16,26 +16,26 @@ */ -package io.pixelsdb.pixels.sink; +package io.pixelsdb.pixels.sink.source; import io.debezium.engine.DebeziumEngine; import io.debezium.engine.RecordChangeEvent; import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.deserializer.RowChangeEventStructDeserializer; -import io.pixelsdb.pixels.sink.deserializer.TransactionStructMessageDeserializer; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.event.TableEnginePipelineManager; -import io.pixelsdb.pixels.sink.event.TablePipelineManager; -import io.pixelsdb.pixels.sink.event.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.event.deserializer.TransactionStructMessageDeserializer; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.processor.MetricsFacade; import io.pixelsdb.pixels.sink.processor.StoppableProcessor; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; +import io.pixelsdb.pixels.sink.provider.TransactionEventEngineProvider; import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; import io.pixelsdb.pixels.sink.sink.ProtoWriter; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; @@ -52,27 +52,27 @@ */ public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer>, StoppableProcessor { - PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); private final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); private final String checkTransactionTopic; private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; - private final TableEnginePipelineManager tableEnginePipelineManager = new TableEnginePipelineManager(); + private final TableProviderAndProcessorPipelineManager tableProvidersManagerImpl = new TableProviderAndProcessorPipelineManager<>(); private final TransactionProcessor processor = new TransactionProcessor(transactionEventProvider); - private final Thread processorThread; private final Thread adapterThread; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final PixelsSinkMode pixelsSinkMode; private final ProtoWriter protoWriter; + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + public PixelsDebeziumConsumer() { this.checkTransactionTopic = pixelsSinkConfig.getDebeziumTopicPrefix() + ".transaction"; - adapterThread = new Thread(this::processTransactionSourceRecord, "transaction-adapter"); + adapterThread = new Thread(this.transactionEventProvider, "transaction-adapter"); adapterThread.start(); - processorThread = new Thread(processor, "debezium-processor"); + Thread processorThread = new Thread(processor, "debezium-processor"); processorThread.start(); pixelsSinkMode = pixelsSinkConfig.getPixelsSinkMode(); - if(pixelsSinkMode == PixelsSinkMode.PROTO) + if (pixelsSinkMode == PixelsSinkMode.PROTO) { try { @@ -89,19 +89,20 @@ public PixelsDebeziumConsumer() public void handleBatch(List> event, - DebeziumEngine.RecordCommitter> committer) throws InterruptedException { - for(RecordChangeEvent record:event) + DebeziumEngine.RecordCommitter> committer) throws InterruptedException + { + for (RecordChangeEvent record : event) { try { SourceRecord sourceRecord = record.record(); - if(sourceRecord == null) + if (sourceRecord == null) { continue; } metricsFacade.recordDebeziumEvent(); - if(isTransactionEvent(sourceRecord)) + if (isTransactionEvent(sourceRecord)) { switch (pixelsSinkMode) { @@ -120,7 +121,8 @@ public void handleBatch(List> event, throw new RuntimeException("Sink Mode " + pixelsSinkMode.toString() + "is not supported"); } } - } else { + } else + { switch (pixelsSinkMode) { case RETINA -> @@ -132,7 +134,7 @@ public void handleBatch(List> event, try { RowChangeEvent rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(sourceRecord); - protoWriter.write(rowChangeEvent); + protoWriter.writeRow(rowChangeEvent); metricsFacade.recordRowEvent(); } catch (SinkException e) { @@ -146,8 +148,7 @@ public void handleBatch(List> event, } } } - } - finally + } finally { committer.markProcessed(record); } @@ -168,24 +169,7 @@ private void handleRowChangeSourceRecord(SourceRecord sourceRecord) String schemaName = source.get("db").toString(); String tableName = source.get("table").toString(); SchemaTableName schemaTableName = new SchemaTableName(schemaName, tableName); - tableEnginePipelineManager.routeRecord(schemaTableName, sourceRecord); - } - - private void processTransactionSourceRecord() { - while (true) { - try { - SourceRecord sourceRecord = rawTransactionQueue.take(); - SinkProto.TransactionMetadata tx = transactionEventProvider.convert(sourceRecord); - if (tx != null) { - transactionEventProvider.getEventQueue().put(tx); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } catch (Exception e) { - throw new RuntimeException(e); - } - } + tableProvidersManagerImpl.routeRecord(schemaTableName, sourceRecord); } private boolean isTransactionEvent(SourceRecord sourceRecord) @@ -198,7 +182,7 @@ public void stopProcessor() { adapterThread.interrupt(); processor.stopProcessor(); - if(protoWriter != null) + if (protoWriter != null) { try { diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java similarity index 84% rename from src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java rename to src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java index a02994c..33cbb6a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkEngineProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java @@ -1,4 +1,4 @@ -package io.pixelsdb.pixels.sink.processor;/* +package io.pixelsdb.pixels.sink.source;/* * Copyright 2025 PixelsDB. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -20,7 +20,6 @@ import io.debezium.engine.DebeziumEngine; import io.debezium.engine.RecordChangeEvent; import io.debezium.engine.format.ChangeEventFormat; -import io.pixelsdb.pixels.sink.PixelsDebeziumConsumer; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import org.apache.kafka.connect.source.SourceRecord; @@ -30,21 +29,24 @@ /** * @package: PACKAGE_NAME - * @className: io.pixelsdb.pixels.sink.processor.SinkEngineProcessor + * @className: io.pixelsdb.pixels.sink.source.SinkEngineSource * @author: AntiO2 * @date: 2025/9/25 09:17 */ -public class SinkEngineProcessor implements MainProcessor +public class SinkEngineSource implements SinkSource { private final PixelsDebeziumConsumer consumer; private DebeziumEngine> engine; private ExecutorService executor; private volatile boolean running = true; - public SinkEngineProcessor() { + + public SinkEngineSource() + { this.consumer = new PixelsDebeziumConsumer(); } - public void start() { + public void start() + { Properties debeziumProps = PixelsSinkConfigFactory.getInstance() .getConfig().extractPropertiesByPrefix("debezium.", true); @@ -60,15 +62,19 @@ public void start() { @Override public void stopProcessor() { - try { - if (engine != null) { + try + { + if (engine != null) + { engine.close(); } - if (executor != null) { + if (executor != null) + { executor.shutdown(); } consumer.stopProcessor(); - } catch (Exception e) { + } catch (Exception e) + { throw new RuntimeException("Failed to stop PixelsSinkEngine", e); } finally { diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkKafkaProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java similarity index 88% rename from src/main/java/io/pixelsdb/pixels/sink/processor/SinkKafkaProcessor.java rename to src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java index 48ef519..4f54742 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/SinkKafkaProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java @@ -15,18 +15,19 @@ * */ -package io.pixelsdb.pixels.sink.processor; +package io.pixelsdb.pixels.sink.source; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import io.pixelsdb.pixels.sink.config.factory.KafkaPropFactorySelector; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.prometheus.client.exporter.HTTPServer; +import io.pixelsdb.pixels.sink.processor.MonitorThreadManager; +import io.pixelsdb.pixels.sink.processor.TopicProcessor; +import io.pixelsdb.pixels.sink.processor.TransactionProcessor; -import java.io.IOException; import java.util.Properties; -public class SinkKafkaProcessor implements MainProcessor +public class SinkKafkaSource implements SinkSource { private MonitorThreadManager manager; private volatile boolean running = true; @@ -50,7 +51,6 @@ public void start() manager = new MonitorThreadManager(); manager.startMonitor(transactionProcessor); manager.startMonitor(topicMonitor); - } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java similarity index 75% rename from src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java rename to src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java index 5bb3d02..16a736b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MainProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java @@ -15,17 +15,20 @@ * */ -package io.pixelsdb.pixels.sink.processor; +package io.pixelsdb.pixels.sink.source; +import io.pixelsdb.pixels.sink.processor.StoppableProcessor; + /** * @package: io.pixelsdb.pixels.sink.processor - * @className: MainProcessor + * @className: SinkSource * @author: AntiO2 * @date: 2025/9/26 13:45 */ -public interface MainProcessor extends StoppableProcessor +public interface SinkSource extends StoppableProcessor { - public void start(); - public boolean isRunning(); + void start(); + + boolean isRunning(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java new file mode 100644 index 0000000..fde204d --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java @@ -0,0 +1,36 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.source; + +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; + +public class SinkSourceFactory +{ + public static SinkSource createSinkSource() + { + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + return switch (config.getDataSource()) + { + case "kafka" -> new SinkKafkaSource(); + case "engine" -> new SinkEngineSource(); + case "storage" -> new FasterSinkStorageSource(); + default -> throw new IllegalStateException("Unsupported data source type: " + config.getDataSource()); + }; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java index 3ae09ea..8f3206c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java @@ -21,74 +21,92 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import io.etcd.jetcd.KeyValue; import io.pixelsdb.pixels.common.utils.EtcdUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import java.util.stream.Collectors; -import io.etcd.jetcd.KeyValue; - /** * @package: io.pixelsdb.pixels.sink.util * @className: EtcdFileRegistry * @author: AntiO2 * @date: 2025/10/5 08:24 */ -public class EtcdFileRegistry { +public class EtcdFileRegistry +{ private static final Logger LOGGER = LoggerFactory.getLogger(EtcdFileRegistry.class); private static final String REGISTRY_PREFIX = "/sink/proto/registry/"; - + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private final String topic; private final String baseDir; private final EtcdUtil etcd = EtcdUtil.Instance(); private final AtomicInteger nextFileId = new AtomicInteger(0); private String currentFileKey; - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public EtcdFileRegistry(String topic, String baseDir) { + public EtcdFileRegistry(String topic, String baseDir) + { this.topic = topic; this.baseDir = baseDir; initRegistry(); } - private void initRegistry() { + public static String extractPath(String etcdValue) + { + try + { + Map meta = OBJECT_MAPPER.readValue(etcdValue, Map.class); + return (String) meta.get("path"); + } catch (IOException e) + { + LOGGER.error("Failed to parse etcd value: {}", etcdValue, e); + return null; + } + } + + private void initRegistry() + { List files = etcd.getKeyValuesByPrefix(filePrefix()); - if (!files.isEmpty()) { + if (!files.isEmpty()) + { int maxId = files.stream() .mapToInt(kv -> extractFileId(kv.getKey().toString())) .max() .orElse(0); nextFileId.set(maxId + 1); LOGGER.info("Initialized registry for topic {} with nextFileId={}", topic, nextFileId.get()); - } else { + } else + { LOGGER.info("No existing files found for topic {}, starting fresh", topic); } } - private String topicPrefix() { + private String topicPrefix() + { return REGISTRY_PREFIX + topic; } - private String filePrefix() { + private String filePrefix() + { return topicPrefix() + "/files/"; } - private int extractFileId(String key) { - try { + private int extractFileId(String key) + { + try + { String fileName = key.substring(key.lastIndexOf('/') + 1); String id = fileName.replace(".proto", ""); return Integer.parseInt(id); - } catch (Exception e) { + } catch (Exception e) + { return 0; } } @@ -96,7 +114,8 @@ private int extractFileId(String key) { /** * Create a new file and register it in etcd. */ - public synchronized String createNewFile() { + public synchronized String createNewFile() + { String fileName = String.format("%05d.proto", nextFileId.getAndIncrement()); String fullPath = baseDir + "/" + topic + "/" + fileName; @@ -122,29 +141,20 @@ public synchronized String createNewFile() { return fullPath; } - public synchronized String getCurrentFileKey() { + public synchronized String getCurrentFileKey() + { return currentFileKey; } - public static String extractPath(String etcdValue) - { - try - { - Map meta = OBJECT_MAPPER.readValue(etcdValue, Map.class); - return (String) meta.get("path"); - } catch (IOException e) - { - LOGGER.error("Failed to parse etcd value: {}", etcdValue, e); - return null; - } - } /** * List all files (for readers). */ - public List listAllFiles() { + public List listAllFiles() + { List files = etcd.getKeyValuesByPrefix(filePrefix()); return files.stream() - .map(kv -> { + .map(kv -> + { String value = kv.getValue().toString(); return extractPath(value); }) @@ -155,7 +165,8 @@ public List listAllFiles() { /** * Mark a file as completed (for writer rotation). */ - public void markFileCompleted(String fileName) { + public void markFileCompleted(String fileName) + { KeyValue kv = etcd.getKeyValue(fileName); if (kv == null) return; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java similarity index 95% rename from src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java rename to src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index e6b9922..463bf75 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.processor; +package io.pixelsdb.pixels.sink.util; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -34,8 +34,8 @@ public class MetricsFacade { - private static final Logger LOGGER = LoggerFactory.getLogger(MetricsFacade.class); - + private static final Logger LOGGER = LoggerFactory.getLogger(MetricsFacade.class); + private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); private static MetricsFacade instance; private final boolean enabled; private final Counter tableChangeCounter; @@ -47,15 +47,11 @@ public class MetricsFacade private final Counter rawDataThroughputCounter; private final Counter debeziumEventCounter; private final Counter rowEventCounter; - private final Summary transServiceLatency; private final Summary indexServiceLatency; private final Summary retinaServiceLatency; private final Summary writerLatency; private final Summary totalLatency; - - private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - private final boolean monitorReportEnabled; private final int monitorReportInterval; private final String monitorReportPath; @@ -193,25 +189,17 @@ private MetricsFacade(boolean enabled) monitorReportInterval = config.getMonitorReportInterval(); monitorReportPath = config.getMonitorReportFile(); - if(monitorReportEnabled) + if (monitorReportEnabled) { running.set(true); reportThread = new Thread(this::run, "Metrics Report Thread"); LOGGER.info("Metrics Report Thread Started"); reportThread.start(); - } else { + } else + { reportThread = null; } } - public void stop() - { - running.set(false); - if (reportThread != null) - { - reportThread.interrupt(); - } - LOGGER.info("Monitor report thread stopped."); - } private static synchronized void initialize() { @@ -230,9 +218,19 @@ public static MetricsFacade getInstance() return instance; } + public void stop() + { + running.set(false); + if (reportThread != null) + { + reportThread.interrupt(); + } + LOGGER.info("Monitor report thread stopped."); + } + public void recordDebeziumEvent() { - if(enabled && debeziumEventCounter != null) + if (enabled && debeziumEventCounter != null) { debeziumEventCounter.inc(); } @@ -364,13 +362,11 @@ public void run() { logPerformance(); Thread.sleep(monitorReportInterval); - } - catch (InterruptedException e) + } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; - } - catch (Throwable t) + } catch (Throwable t) { LOGGER.warn("Error while reporting performance.", t); } @@ -412,18 +408,20 @@ public void logPerformance() " in {} ms", deltaRows, String.format("%.2f", rowOips), deltaTxns, String.format("%.2f", txnOips), - deltaDebezium, String.format("%.2f", dbOips), - deltaSerdRows, String.format("%.2f", serdRowsOips), - deltaSerdTxs, String.format("%.2f", serdTxsOips), + deltaDebezium, String.format("%.2f", dbOips), + deltaSerdRows, String.format("%.2f", serdRowsOips), + deltaSerdTxs, String.format("%.2f", serdTxsOips), monitorReportInterval ); String time = LocalDateTime.now().format(DateTimeFormatter.ofPattern("HH:mm:ss")); // Append to CSV for plotting - try (FileWriter fw = new FileWriter(monitorReportPath, true)) { + try (FileWriter fw = new FileWriter(monitorReportPath, true)) + { fw.write(String.format("%s,%.2f,%.2f,%.2f,%.2f,%.2f%n", time, rowOips, txnOips, dbOips, serdRowsOips, serdTxsOips)); - } catch (IOException e) { + } catch (IOException e) + { LOGGER.warn("Failed to write perf metrics: " + e.getMessage()); } } diff --git a/src/main/resources/log4j2.properties b/src/main/resources/log4j2.properties index 72a2f4d..dcc5c79 100644 --- a/src/main/resources/log4j2.properties +++ b/src/main/resources/log4j2.properties @@ -15,7 +15,7 @@ appender.rolling.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n rootLogger.level=info rootLogger.appenderRef.stdout.ref=STDOUT rootLogger.appenderRef.log.ref=log -logger.transaction.name=io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator +logger.transaction.name=io.pixelsdb.pixels.sink.sink.retina.RetinaWriter logger.transaction.level=info logger.transaction.appenderRef.log.ref=log logger.transaction.appenderRef.stdout.ref=STDOUT diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties index f77131e..de99b69 100644 --- a/src/main/resources/pixels-sink.aws.properties +++ b/src/main/resources/pixels-sink.aws.properties @@ -8,12 +8,11 @@ group.id=3078 auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer #value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer -value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer +value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config topic.prefix=postgresql.oltp_server consumer.capture_database=pixels_bench_sf1x consumer.include_tables= - sink.csv.path=./data sink.csv.enable_header=false ## Retina Config @@ -27,23 +26,20 @@ sink.timeout.ms=5000 sink.flush.interval.ms=100 sink.flush.batch.size=500 sink.max.retries=3 - ## sink commit sink.commit.batch.size=500 sink.commit.batch.worker=32 sink.commit.batch.delay=200 - ## Proto Config sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp sink.proto.data=data sink.proto.maxRecords=1000000 - ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction #transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer -transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=200 ## Batch or trans or record sink.trans.mode=batch @@ -55,7 +51,6 @@ sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/ParallelRocksDB2.csv # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 - # debezium debezium.name=testEngine debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector diff --git a/src/main/resources/pixels-sink.local.properties b/src/main/resources/pixels-sink.local.properties index 1dc91ac..360cfb1 100644 --- a/src/main/resources/pixels-sink.local.properties +++ b/src/main/resources/pixels-sink.local.properties @@ -8,12 +8,11 @@ group.id=3107 auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer #value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer -value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer +value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config topic.prefix=postgresql.oltp_server consumer.capture_database=pixels_bench_sf1x consumer.include_tables= - sink.csv.path=./data sink.csv.enable_header=false ## Retina Config @@ -28,23 +27,20 @@ sink.timeout.ms=5000 sink.flush.interval.ms=100 sink.flush.batch.size=100 sink.max.retries=3 - ## sink commit sink.commit.batch.size=500 sink.commit.batch.worker=16 sink.commit.batch.delay=200 - ## Proto Config sink.proto.dir=file:///home/pixels/projects/pixels-sink/tmp sink.proto.data=data sink.proto.maxRecords=1000000 - ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction #transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer -transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 ## Batch or trans or record sink.trans.mode=batch @@ -54,7 +50,6 @@ sink.monitor.port=9464 # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 - # debezium debezium.name=testEngine debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector diff --git a/src/main/resources/pixels-sink.properties b/src/main/resources/pixels-sink.properties index 2bad52b..d561fb6 100644 --- a/src/main/resources/pixels-sink.properties +++ b/src/main/resources/pixels-sink.properties @@ -3,7 +3,7 @@ bootstrap.servers=pixels_kafka:9092 group.id=docker-pixels-table auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer -value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer +value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventAvroDeserializer #value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config topic.prefix=oltp_server @@ -23,7 +23,7 @@ sink.max.retries=3 sink.registry.url=http://apicurio:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction -transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionAvroMessageDeserializer #transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 ## batch or record diff --git a/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java b/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java index 4e8922b..58659ff 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/DebeziumEngineTest.java @@ -19,7 +19,6 @@ package io.pixelsdb.pixels.sink; -import io.debezium.config.Configuration; import io.debezium.embedded.Connect; import io.debezium.engine.DebeziumEngine; import io.debezium.engine.RecordChangeEvent; @@ -27,7 +26,6 @@ import org.apache.kafka.connect.source.SourceRecord; import org.junit.jupiter.api.Test; -import javax.swing.event.ChangeEvent; import java.util.List; import java.util.Properties; import java.util.concurrent.ExecutorService; @@ -84,13 +82,16 @@ public void testPostgresCDC() ExecutorService executor = Executors.newSingleThreadExecutor(); executor.execute(engine); - while (true) { + while (true) + { } } - class MyChangeConsumer implements DebeziumEngine.ChangeConsumer> { - public void handleBatch(List> event, DebeziumEngine.RecordCommitter> committer) throws InterruptedException { + class MyChangeConsumer implements DebeziumEngine.ChangeConsumer> + { + public void handleBatch(List> event, DebeziumEngine.RecordCommitter> committer) throws InterruptedException + { committer.markBatchFinished(); } } diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java similarity index 64% rename from src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java rename to src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java index 74eee8a..4cc712a 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionCoordinatorTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java @@ -22,6 +22,8 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.sink.retina.RetinaWriter; +import io.pixelsdb.pixels.sink.sink.retina.SinkContext; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -42,10 +44,10 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -class TransactionCoordinatorTest +class RetinaWriterTest { - private static final Logger LOGGER = LoggerFactory.getLogger(TransactionCoordinatorTest.class); - private TransactionCoordinator coordinator; + private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriterTest.class); + private RetinaWriter coordinator; private List dispatchedEvents; private ExecutorService testExecutor; private CountDownLatch latch; @@ -57,11 +59,11 @@ void setUp() throws IOException testExecutor = TestUtils.synchronousExecutor(); dispatchedEvents = Collections.synchronizedList(new ArrayList<>()); - coordinator = new TestableCoordinator(dispatchedEvents); + coordinator = new RetinaWriter(); try { - Field executorField = TransactionCoordinator.class + Field executorField = RetinaWriter.class .getDeclaredField("dispatchExecutor"); executorField.setAccessible(true); executorField.set(coordinator, testExecutor); @@ -109,11 +111,11 @@ private RowChangeEvent buildEvent(String txId, String table, long collectionOrde @Test void shouldProcessOrderedEvents() throws Exception { - coordinator.processTransactionEvent(buildBeginTx("tx1")); + coordinator.writeTrans(buildBeginTx("tx1")); - coordinator.processRowEvent(buildEvent("tx1", "orders", 1, 1)); - coordinator.processRowEvent(buildEvent("tx1", "orders", 2, 2)); - coordinator.processTransactionEvent(buildEndTx("tx1")); + coordinator.writeRow(buildEvent("tx1", "orders", 1, 1)); + coordinator.writeRow(buildEvent("tx1", "orders", 2, 2)); + coordinator.writeTrans(buildEndTx("tx1")); assertEquals(2, dispatchedEvents.size()); assertTrue(dispatchedEvents.get(0).contains("Order: 1/1")); @@ -123,11 +125,11 @@ void shouldProcessOrderedEvents() throws Exception @Test void shouldHandleOutOfOrderEvents() throws SinkException { - coordinator.processTransactionEvent(buildBeginTx("tx2")); - coordinator.processRowEvent(buildEvent("tx2", "users", 3, 3)); - coordinator.processRowEvent(buildEvent("tx2", "users", 2, 2)); - coordinator.processRowEvent(buildEvent("tx2", "users", 1, 1)); - coordinator.processTransactionEvent(buildEndTx("tx2")); + coordinator.writeTrans(buildBeginTx("tx2")); + coordinator.writeRow(buildEvent("tx2", "users", 3, 3)); + coordinator.writeRow(buildEvent("tx2", "users", 2, 2)); + coordinator.writeRow(buildEvent("tx2", "users", 1, 1)); + coordinator.writeTrans(buildEndTx("tx2")); assertTrue(dispatchedEvents.get(0).contains("Order: 1/1")); assertTrue(dispatchedEvents.get(1).contains("Order: 2/2")); assertTrue(dispatchedEvents.get(2).contains("Order: 3/3")); @@ -136,9 +138,9 @@ void shouldHandleOutOfOrderEvents() throws SinkException @Test void shouldRecoverOrphanedEvents() throws SinkException { - coordinator.processRowEvent(buildEvent("tx3", "logs", 1, 1)); // orphan event - coordinator.processTransactionEvent(buildBeginTx("tx3")); // recover - coordinator.processTransactionEvent(buildEndTx("tx3")); + coordinator.writeRow(buildEvent("tx3", "logs", 1, 1)); // orphan event + coordinator.writeTrans(buildBeginTx("tx3")); // recover + coordinator.writeTrans(buildEndTx("tx3")); assertTrue(dispatchedEvents.get(0).contains("Order: 1/1")); } @@ -151,25 +153,12 @@ void shouldProcessNonTransactionalEvents(SinkProto.OperationType opType) throws .setOp(opType) .build(), null ); - coordinator.processRowEvent(event); + coordinator.writeRow(event); TimeUnit.MILLISECONDS.sleep(10); assertEquals(1, dispatchedEvents.size()); PixelsSinkConfigFactory.reset(); } - @Test - void shouldHandleTransactionTimeout() throws Exception - { - TransactionCoordinator fastTimeoutCoordinator = new TransactionCoordinator(); - fastTimeoutCoordinator.setTxTimeoutMs(100); - - fastTimeoutCoordinator.processTransactionEvent(buildBeginTx("tx4")); - fastTimeoutCoordinator.processRowEvent(buildEvent("tx4", "temp", 1, 1)); - - TimeUnit.MILLISECONDS.sleep(150); // wait for timeout - assertEquals(1, fastTimeoutCoordinator.activeTxContexts.size()); - } - @ParameterizedTest @ValueSource(ints = {1, 3, 9, 16}) void shouldHandleConcurrentEvents(int threadCount) throws SinkException, IOException, InterruptedException @@ -178,7 +167,7 @@ void shouldHandleConcurrentEvents(int threadCount) throws SinkException, IOExcep PixelsSinkConfigFactory.initialize(""); latch = new CountDownLatch(threadCount); - coordinator.processTransactionEvent(buildBeginTx("tx5")); + coordinator.writeTrans(buildBeginTx("tx5")); // concurrently send event for (int i = 0; i < threadCount; i++) { @@ -187,7 +176,7 @@ void shouldHandleConcurrentEvents(int threadCount) throws SinkException, IOExcep { try { - coordinator.processRowEvent(buildEvent("tx5", "concurrent", order, order)); + coordinator.writeRow(buildEvent("tx5", "concurrent", order, order)); } catch (SinkException e) { throw new RuntimeException(e); @@ -198,7 +187,7 @@ void shouldHandleConcurrentEvents(int threadCount) throws SinkException, IOExcep assertTrue(latch.await(1, TimeUnit.SECONDS)); - coordinator.processTransactionEvent(buildEndTx("tx5")); + coordinator.writeTrans(buildEndTx("tx5")); LOGGER.debug("Thread Count: {} DispatchedEvents size: {}", threadCount, dispatchedEvents.size()); LOGGER.debug("Thread Count: {} DispatchedEvents size: {}", threadCount, dispatchedEvents.size()); @@ -206,47 +195,4 @@ void shouldHandleConcurrentEvents(int threadCount) throws SinkException, IOExcep assertEquals(threadCount, dispatchedEvents.size()); PixelsSinkConfigFactory.reset(); } - - - private static class TestableCoordinator extends TransactionCoordinator - { - private static final Logger LOGGER = LoggerFactory.getLogger(TestableCoordinator.class); - private final List eventLog; - - TestableCoordinator(List eventLog) - { - this.eventLog = eventLog; - } - - @Override - protected void dispatchImmediately(RowChangeEvent event, SinkContext ctx) - { - dispatchExecutor.execute(() -> - { - try - { - String log = String.format("Dispatching [%s] %s.%s (Order: %s/%s)", - event.getOp().name(), - event.getDb(), - event.getTable(), - event.getTransaction() != null ? - event.getTransaction().getDataCollectionOrder() : "N/A", - event.getTransaction() != null ? - event.getTransaction().getTotalOrder() : "N/A"); - LOGGER.info(log); - eventLog.add(log); - LOGGER.debug("Event log size : {}", eventLog.size()); - } finally - { - if (ctx != null) - { - if (ctx.pendingEvents.decrementAndGet() == 0 && ctx.completed) - { - ctx.completionFuture.complete(null); - } - } - } - }); - } - } } \ No newline at end of file diff --git a/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java b/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java index a459f79..3ee867d 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/consumer/AvroConsumerTest.java @@ -23,8 +23,8 @@ import io.apicurio.registry.serde.avro.AvroKafkaDeserializer; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventAvroDeserializer; import io.pixelsdb.pixels.sink.exception.SinkException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java similarity index 96% rename from src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java rename to src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java index d9b2e5f..ca20e54 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowBatchTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java @@ -1,4 +1,4 @@ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.core.vector.BinaryColumnVector; diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java similarity index 95% rename from src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java rename to src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java index 32caee2..7eb958c 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowChangeEventDeserializerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java @@ -15,9 +15,10 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.junit.jupiter.api.Test; diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParserTest.java similarity index 97% rename from src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java rename to src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParserTest.java index ffc1b6e..0eee54b 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/RowDataParserTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParserTest.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.sink.util.DateUtil; import org.junit.jupiter.api.Test; diff --git a/src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java similarity index 97% rename from src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java rename to src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java index b6270d0..4150ebb 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/deserializer/SchemaDeserializerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java @@ -15,11 +15,12 @@ * */ -package io.pixelsdb.pixels.sink.deserializer; +package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import io.pixelsdb.pixels.core.TypeDescription; +import io.pixelsdb.pixels.sink.event.deserializer.SchemaDeserializer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java index 424c1c2..cc2fd89 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java @@ -21,13 +21,8 @@ import com.google.protobuf.ByteString; import io.pixelsdb.pixels.common.physical.*; -import io.pixelsdb.pixels.common.retina.RetinaService; -import io.pixelsdb.pixels.common.transaction.TransService; -import io.pixelsdb.pixels.daemon.TransProto; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; import io.pixelsdb.pixels.storage.localfs.PhysicalLocalReader; import lombok.SneakyThrows; import org.junit.jupiter.api.BeforeAll; @@ -55,6 +50,45 @@ public static void setUp() throws IOException PixelsSinkConfigFactory.initialize("/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties"); // PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties"); } + + private static SinkProto.RowRecord getRowRecord(int i) + { + byte[][] cols = new byte[3][]; + + cols[0] = Integer.toString(i).getBytes(StandardCharsets.UTF_8); + cols[1] = Long.toString(i * 1000L).getBytes(StandardCharsets.UTF_8); + cols[2] = ("row_" + i).getBytes(StandardCharsets.UTF_8); + SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); + afterValueBuilder + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).build()) + .addValues( + SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).build()); + + + SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); + builder.setOp(SinkProto.OperationType.INSERT) + .setAfter(afterValueBuilder) + .setSource( + SinkProto.SourceInfo.newBuilder() + .setDb(schemaName) + .setTable(tableName) + .build() + ); + return builder.build(); + } + + private static SinkProto.TransactionMetadata getTrans(int i, SinkProto.TransactionStatus status) + { + SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); + builder.setId(Integer.toString(i)); + builder.setStatus(status); + builder.setTimestamp(System.currentTimeMillis()); + return builder.build(); + } + @SneakyThrows @Test public void testWriteTransInfo() @@ -79,7 +113,7 @@ public void testWriteFile() throws IOException int writeNum = 3; ByteBuffer buf = ByteBuffer.allocate(writeNum * Integer.BYTES); - for(int i = 0; i < 3; i++) + for (int i = 0; i < 3; i++) { buf.putInt(i); } @@ -87,7 +121,6 @@ public void testWriteFile() throws IOException writer.close(); } - @Test public void testReadFile() throws IOException { @@ -95,11 +128,12 @@ public void testReadFile() throws IOException PhysicalLocalReader reader = (PhysicalLocalReader) PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.file, path); int writeNum = 12; - for(int i = 0; i < writeNum; i++) + for (int i = 0; i < writeNum; i++) { reader.readLong(ByteOrder.BIG_ENDIAN); } } + @Test public void testReadEmptyFile() throws IOException { @@ -120,7 +154,7 @@ public void testWriteRowInfo() throws IOException for (int i = 0; i < maxTx; i++) { transWriter.writeTrans(getTrans(i, SinkProto.TransactionStatus.BEGIN)); - for(int j = i; j < 3; j++) + for (int j = i; j < 3; j++) { transWriter.write(getRowRecord(rowCnt++)); } @@ -128,42 +162,4 @@ public void testWriteRowInfo() throws IOException } transWriter.close(); } - - private static SinkProto.RowRecord getRowRecord(int i) - { - byte[][] cols = new byte[3][]; - - cols[0] = Integer.toString(i).getBytes(StandardCharsets.UTF_8); - cols[1] = Long.toString(i * 1000L).getBytes(StandardCharsets.UTF_8); - cols[2] = ("row_" + i).getBytes(StandardCharsets.UTF_8); - SinkProto.RowValue.Builder afterValueBuilder = SinkProto.RowValue.newBuilder(); - afterValueBuilder - .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[0]))).build()) - .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[1]))).build()) - .addValues( - SinkProto.ColumnValue.newBuilder().setValue(ByteString.copyFrom((cols[2]))).build()); - - - SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); - builder.setOp(SinkProto.OperationType.INSERT) - .setAfter(afterValueBuilder) - .setSource( - SinkProto.SourceInfo.newBuilder() - .setDb(schemaName) - .setTable(tableName) - .build() - ); - return builder.build(); - } - - private static SinkProto.TransactionMetadata getTrans(int i, SinkProto.TransactionStatus status) - { - SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); - builder.setId(Integer.toString(i)); - builder.setStatus(status); - builder.setTimestamp(System.currentTimeMillis()); - return builder.build(); - } } diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java index a8aa9b1..0cc902c 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java @@ -6,15 +6,15 @@ import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.common.transaction.TransService; -import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.index.IndexProto; import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.concurrent.TransactionManager; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.sink.retina.RetinaServiceProxy; +import io.pixelsdb.pixels.sink.sink.retina.TransactionProxy; import io.pixelsdb.pixels.sink.util.DateUtil; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -188,9 +188,9 @@ public void testCheckingAccountInsertPerformance() throws String schemaName = "pixels_bench_sf1x"; String tableName = "savingaccount"; - PixelsSinkWriter writer = PixelsSinkWriterFactory.getWriter(); + RetinaServiceProxy writer = new RetinaServiceProxy(); - TransactionManager manager = TransactionManager.Instance(); + TransactionProxy manager = TransactionProxy.Instance(); // Step 1: Insert 10,000 records int totalInserts = retinaPerformanceTestMaxId; int batchSize = 5; @@ -202,13 +202,13 @@ public void testCheckingAccountInsertPerformance() throws List> futures = new ArrayList<>(); - for (int b = 0; b < batchCount;) + for (int b = 0; b < batchCount; ) { List tableUpdateData = new ArrayList<>(); - for(int sb = 0; sb < samllBatchCount; sb++) + for (int sb = 0; sb < samllBatchCount; sb++) { ++b; - TransContext ctx = manager.getTransContext(); + TransContext ctx = manager.getNewTransContext(); long timeStamp = ctx.getTimestamp(); RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder = @@ -283,7 +283,7 @@ public void testCheckingAccountInsertPerformance() throws futures.add(future); } Assertions.assertNotNull(writer); - if (!writer.writeBatch(schemaName, tableUpdateData)) + if (!writer.writeTrans(schemaName, tableUpdateData)) { logger.error("Error Write Trans"); System.exit(-1); @@ -319,14 +319,14 @@ public void testCheckingAccountUpdatePerformance() throws int clientCount = 2; // 可自定义客户端数量 ExecutorService clientExecutor = Executors.newFixedThreadPool(clientCount); - List writers = new ArrayList<>(); + List writers = new ArrayList<>(); for (int c = 0; c < clientCount; c++) { - writers.add(PixelsSinkWriterFactory.getWriter()); + writers.add(new RetinaServiceProxy()); } Random random = new Random(); - TransactionManager manager = TransactionManager.Instance(); + TransactionProxy manager = TransactionProxy.Instance(); long start = System.currentTimeMillis(); List> futures = new ArrayList<>(); @@ -339,9 +339,9 @@ public void testCheckingAccountUpdatePerformance() throws try { // 轮询选择客户端 - PixelsSinkWriter writer = writers.get(batchIndex % clientCount); + RetinaServiceProxy writer = writers.get(batchIndex % clientCount); - TransContext ctx = manager.getTransContext(); + TransContext ctx = manager.getNewTransContext(); long timeStamp = ctx.getTimestamp(); List tableUpdateData = new ArrayList<>(); @@ -403,7 +403,7 @@ public void testCheckingAccountUpdatePerformance() throws tableUpdateData.add(tableUpdateDataBuilder.build()); long startTime = System.currentTimeMillis(); - if (!writer.writeTrans(schemaName, tableUpdateData, timeStamp)) + if (!writer.writeTrans(schemaName, tableUpdateData)) { logger.error("Error Write Trans"); System.exit(-1); @@ -448,7 +448,7 @@ public void testInsertTwoTablePerformance() throws String tableName2 = "savingaccount"; PixelsSinkWriter writer = PixelsSinkWriterFactory.getWriter(); - TransactionManager manager = TransactionManager.Instance(); + TransactionProxy manager = TransactionProxy.Instance(); // Step 1: Insert 10,000 records int totalInserts = retinaPerformanceTestRowCount; int batchSize = 50; @@ -461,7 +461,7 @@ public void testInsertTwoTablePerformance() throws for (int b = 0; b < batchCount; b++) { - TransContext ctx = manager.getTransContext(); + TransContext ctx = manager.getNewTransContext(); long timeStamp = ctx.getTimestamp(); List tableUpdateData = new ArrayList<>(); @@ -549,7 +549,7 @@ public void testInsertTwoTablePerformance() throws try { // 执行原始的 writeTrans 方法 - writer.writeTrans(schemaName, tableUpdateData, timeStamp); + // writer.writeTrans(schemaName, tableUpdateData, timeStamp); // 记录结束时间 long endTime = System.currentTimeMillis(); diff --git a/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java b/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java index dee5868..aee4a0e 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistryTest.java @@ -34,17 +34,18 @@ public class EtcdFileRegistryTest { private static final Logger LOGGER = LoggerFactory.getLogger(EtcdFileRegistryTest.class); + @Test public void testCreateFile() { EtcdFileRegistry etcdFileRegistry = new EtcdFileRegistry("test", "file:///tmp/test/ray"); - for(int i = 0; i < 10; i++) + for (int i = 0; i < 10; i++) { String newFile = etcdFileRegistry.createNewFile(); etcdFileRegistry.markFileCompleted(newFile); } List files = etcdFileRegistry.listAllFiles(); - for(String file : files) + for (String file : files) { LOGGER.info(file); } diff --git a/src/test/resources/log4j2.properties b/src/test/resources/log4j2.properties index a7675ed..8d15751 100644 --- a/src/test/resources/log4j2.properties +++ b/src/test/resources/log4j2.properties @@ -15,7 +15,7 @@ appender.rolling.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n rootLogger.level=info rootLogger.appenderRef.stdout.ref=STDOUT rootLogger.appenderRef.log.ref=log -logger.transaction.name=io.pixelsdb.pixels.sink.concurrent.TransactionCoordinator +logger.transaction.name=io.pixelsdb.pixels.sink.sink.retina.RetinaWriter logger.transaction.level=info logger.transaction.appenderRef.log.ref=log logger.transaction.appenderRef.stdout.ref=STDOUT From ad3e8474806be6cd6125195f3b6e404bcbf98562 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sat, 11 Oct 2025 15:49:26 +0800 Subject: [PATCH 07/53] Refactor --- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 2 +- .../pixels/sink/config/ConfigKey.java | 2 +- .../pixels/sink/config/ConfigLoader.java | 6 +- .../pixels/sink/config/PixelsSinkConfig.java | 6 +- .../sink/config/PixelsSinkDefaultConfig.java | 7 -- .../RowChangeEventStructDeserializer.java | 2 +- .../TransactionStructMessageDeserializer.java | 2 +- .../pixels/sink/processor/TableProcessor.java | 5 +- .../pixels/sink/processor/TopicProcessor.java | 2 +- .../sink/processor/TransactionProcessor.java | 4 +- .../pixels/sink/provider/EventProvider.java | 1 - .../pixels/sink/provider/ProtoType.java | 2 +- .../provider/TableEventEngineProvider.java | 2 +- .../sink/provider/TableEventProvider.java | 2 +- ...leProviderAndProcessorPipelineManager.java | 2 +- .../TransactionEventEngineProvider.java | 6 +- .../TransactionEventKafkaProvider.java | 7 +- .../source/AbstractSinkStorageSource.java | 17 ++- .../sink/source/FasterSinkStorageSource.java | 17 +-- .../sink/source/LegacySinkStorageSource.java | 3 +- .../sink/source/PixelsDebeziumConsumer.java | 100 +++--------------- .../pixels/sink/source/SinkEngineSource.java | 6 -- .../pixels/sink/source/SinkSource.java | 2 +- .../sink/{sink => writer}/CsvWriter.java | 2 +- .../sink/{sink => writer}/NoneWriter.java | 11 +- .../sink/{sink => writer}/PixelsSinkMode.java | 4 +- .../{sink => writer}/PixelsSinkWriter.java | 2 +- .../PixelsSinkWriterFactory.java | 4 +- .../sink/{sink => writer}/ProtoWriter.java | 4 +- .../RotatingWriterManager.java | 4 +- .../retina/RetinaServiceProxy.java | 6 +- .../{sink => writer}/retina/RetinaWriter.java | 4 +- .../{sink => writer}/retina/SinkContext.java | 2 +- .../retina/SinkContextManager.java | 2 +- .../retina/TableCrossTxWriter.java | 4 +- .../retina/TableSingleTxWriter.java | 2 +- .../{sink => writer}/retina/TableWriter.java | 4 +- .../retina/TableWriterProxy.java | 2 +- .../retina/TransactionMode.java | 2 +- .../retina/TransactionProxy.java | 2 +- src/main/resources/pixels-sink.aws.properties | 8 +- src/main/resources/pixels-sink.properties | 4 +- .../sink/concurrent/RetinaWriterTest.java | 3 +- .../RowChangeEventDeserializerTest.java | 1 - .../deserializer/SchemaDeserializerTest.java | 1 - .../{sink => writer}/TestProtoWriter.java | 14 +-- .../{sink => writer}/TestRetinaWriter.java | 6 +- .../sink/{sink => writer}/TpcHTest.java | 2 +- 48 files changed, 99 insertions(+), 206 deletions(-) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/CsvWriter.java (99%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/NoneWriter.java (76%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/PixelsSinkMode.java (93%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/PixelsSinkWriter.java (95%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/PixelsSinkWriterFactory.java (95%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/ProtoWriter.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/RotatingWriterManager.java (96%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/RetinaServiceProxy.java (96%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/RetinaWriter.java (99%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/SinkContext.java (99%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/SinkContextManager.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/TableCrossTxWriter.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/TableSingleTxWriter.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/TableWriter.java (98%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/TableWriterProxy.java (97%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/TransactionMode.java (95%) rename src/main/java/io/pixelsdb/pixels/sink/{sink => writer}/retina/TransactionProxy.java (99%) rename src/test/java/io/pixelsdb/pixels/sink/{sink => writer}/TestProtoWriter.java (92%) rename src/test/java/io/pixelsdb/pixels/sink/{sink => writer}/TestRetinaWriter.java (99%) rename src/test/java/io/pixelsdb/pixels/sink/{sink => writer}/TpcHTest.java (99%) diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 99f92bb..5473f34 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -19,10 +19,10 @@ import io.pixelsdb.pixels.sink.config.CommandLineConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.sink.retina.TransactionProxy; import io.pixelsdb.pixels.sink.source.SinkSource; import io.pixelsdb.pixels.sink.source.SinkSourceFactory; import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.pixelsdb.pixels.sink.writer.retina.TransactionProxy; import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.hotspot.DefaultExports; import org.slf4j.Logger; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java index 61c585d..066c028 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java @@ -25,7 +25,7 @@ import java.lang.annotation.Target; /** - * @package: io.pixelsdb.pixels.sink + * @package: io.pixelsdb.pixels.sink.config * @className: ConfigKey * @author: AntiO2 * @date: 2025/9/26 13:04 diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java index f932e45..5f9d760 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java @@ -19,9 +19,9 @@ package io.pixelsdb.pixels.sink.config; -import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; -import io.pixelsdb.pixels.sink.sink.retina.RetinaServiceProxy; -import io.pixelsdb.pixels.sink.sink.retina.TransactionMode; +import io.pixelsdb.pixels.sink.writer.PixelsSinkMode; +import io.pixelsdb.pixels.sink.writer.retina.RetinaServiceProxy; +import io.pixelsdb.pixels.sink.writer.retina.TransactionMode; import java.lang.reflect.Field; import java.util.Properties; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index f8813f6..57eaf5a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -18,9 +18,9 @@ import io.pixelsdb.pixels.common.utils.ConfigFactory; import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer; -import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; -import io.pixelsdb.pixels.sink.sink.retina.RetinaServiceProxy; -import io.pixelsdb.pixels.sink.sink.retina.TransactionMode; +import io.pixelsdb.pixels.sink.writer.PixelsSinkMode; +import io.pixelsdb.pixels.sink.writer.retina.RetinaServiceProxy; +import io.pixelsdb.pixels.sink.writer.retina.TransactionMode; import lombok.Getter; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java index 2464383..0ea3482 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java @@ -29,17 +29,10 @@ public class PixelsSinkDefaultConfig public static final int SINK_THREAD = 32; public static final int SINK_CONSUMER_THREAD = 8; - // sink.remote.host=localhost -// sink.remote.port=229422 -// sink.batch.size=100 -// sink.timeout.ms=5000 -// sink.flush.interval.ms=5000 -// sink.max.retries=3 // Transaction Service public static final int TRANSACTION_BATCH_SIZE = 100; - // REMOTE BUFFER public static final String SINK_REMOTE_HOST = "localhost"; public static final short SINK_REMOTE_PORT = 22942; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java index 0780a3c..9129885 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java @@ -31,7 +31,7 @@ import java.util.logging.Logger; /** - * @package: io.pixelsdb.pixels.sink.deserializer + * @package: io.pixelsdb.pixels.sink.event.deserializer * @className: RowChangeEventStructDeserializer * @author: AntiO2 * @date: 2025/9/26 12:00 diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java index 92d517f..7ad0c1c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java @@ -26,7 +26,7 @@ import org.slf4j.LoggerFactory; /** - * @package: io.pixelsdb.pixels.sink.deserializer + * @package: io.pixelsdb.pixels.sink.event.deserializer * @className: TransactionStructMessageDeserializer * @author: AntiO2 * @date: 2025/9/26 12:42 diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 4344f22..3df44f1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -21,13 +21,12 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.provider.TableEventProvider; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriterFactory; import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriterFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java index 1f4f6da..26138a9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java @@ -45,9 +45,9 @@ public class TopicProcessor implements StoppableProcessor, Runnable private final AtomicBoolean running = new AtomicBoolean(true); private final Map activeTasks = new ConcurrentHashMap<>(); // track row event consumer + private final ExecutorService executorService = Executors.newCachedThreadPool(); private AdminClient adminClient; private Timer timer; - private final ExecutorService executorService = Executors.newCachedThreadPool(); public TopicProcessor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java index 8b2df64..d5b3597 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -20,8 +20,8 @@ import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.provider.TransactionEventProvider; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriterFactory; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriterFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index e19cd9b..c3f63ec 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -17,7 +17,6 @@ package io.pixelsdb.pixels.sink.provider; -import io.pixelsdb.pixels.sink.source.FasterSinkStorageSource; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java index 362fc07..e7f50be 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java @@ -20,7 +20,7 @@ /** - * @package: io.pixelsdb.pixels.sink.event + * @package: io.pixelsdb.pixels.sink.provider * @className: ProtoType * @author: AntiO2 * @date: 2025/10/5 12:56 diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java index 519a2f7..dccaaf5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java @@ -28,7 +28,7 @@ /** - * @package: io.pixelsdb.pixels.sink.event + * @package: io.pixelsdb.pixels.sink.provider * @className: TableEventEngineProvider * @author: AntiO2 * @date: 2025/9/26 10:45 diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java index 89175d6..4600d93 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java @@ -21,7 +21,7 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; /** - * @package: io.pixelsdb.pixels.sink.event + * @package: io.pixelsdb.pixels.sink.provider * @className: TableEventProvider * @author: AntiO2 * @date: 2025/9/26 07:47 diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index d668b4a..1e10c52 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * @package: io.pixelsdb.pixels.sink.event + * @package: io.pixelsdb.pixels.sink.provider * @className: TableProviderAndProcessorPipelineManager * @author: AntiO2 * @date: 2025/9/26 10:44 diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java index 0ed74fb..d440b6d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java @@ -24,12 +24,8 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; -import java.io.IOException; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - /** - * @package: io.pixelsdb.pixels.sink.event + * @package: io.pixelsdb.pixels.sink.provider * @className: TransactionEventEngineProvider * @author: AntiO2 * @date: 2025/9/25 13:20 diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java index 6762168..95681db 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java @@ -27,16 +27,13 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.errors.WakeupException; -import java.io.IOException; import java.time.Duration; import java.util.Collections; import java.util.Properties; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; /** - * @package: io.pixelsdb.pixels.sink.event + * @package: io.pixelsdb.pixels.sink.provider * @className: TransactionEventKafkaProvider * @author: AntiO2 * @date: 2025/9/25 13:40 @@ -70,7 +67,7 @@ public void processLoop() for (ConsumerRecord record : records) { - if(record.value() == null) + if (record.value() == null) { continue; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 3d964e1..261e2c6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -8,7 +8,9 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; -import io.pixelsdb.pixels.sink.provider.*; +import io.pixelsdb.pixels.sink.provider.ProtoType; +import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; +import io.pixelsdb.pixels.sink.provider.TransactionEventStorageProvider; import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; @@ -31,20 +33,17 @@ public abstract class AbstractSinkStorageSource implements SinkSource protected final String baseDir; protected final EtcdFileRegistry etcdFileRegistry; protected final List files; - - protected TransactionEventStorageProvider transactionEventProvider; - protected TransactionProcessor transactionProcessor; - - protected Thread transactionProviderThread; - protected Thread transactionProcessorThread; - + protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); private final Map consumerThreads = new ConcurrentHashMap<>(); private final int maxQueueCapacity = 10000; private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); private final Map>> queueMap = new ConcurrentHashMap<>(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final TableProviderAndProcessorPipelineManager tablePipelineManager = new TableProviderAndProcessorPipelineManager(); + protected TransactionEventStorageProvider transactionEventProvider; + protected TransactionProcessor transactionProcessor; + protected Thread transactionProviderThread; + protected Thread transactionProcessorThread; protected AbstractSinkStorageSource() { diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java index 2d0e513..2dcbfc3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java @@ -20,29 +20,14 @@ import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.common.physical.PhysicalReader; -import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; -import io.pixelsdb.pixels.common.physical.Storage; -import io.pixelsdb.pixels.core.utils.Pair; -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.processor.TransactionProcessor; import io.pixelsdb.pixels.sink.provider.ProtoType; -import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; -import io.pixelsdb.pixels.sink.provider.TransactionEventEngineProvider; -import io.pixelsdb.pixels.sink.provider.TransactionEventStorageProvider; -import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Map; -import java.util.concurrent.*; /** - * @package: io.pixelsdb.pixels.sink.processor + * @package: io.pixelsdb.pixels.sink.source * @className: LegacySinkStorageSource * @author: AntiO2 * @date: 2025/10/5 11:43 diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java index b461a31..c20df77 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java @@ -24,7 +24,6 @@ import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; import io.pixelsdb.pixels.common.physical.Storage; import io.pixelsdb.pixels.core.utils.Pair; -import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; import io.pixelsdb.pixels.sink.provider.ProtoType; @@ -41,7 +40,7 @@ import java.util.concurrent.*; /** - * @package: io.pixelsdb.pixels.sink.processor + * @package: io.pixelsdb.pixels.sink.source * @className: LegacySinkStorageSource * @author: AntiO2 * @date: 2025/10/5 11:43 diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java index b8948ce..e5a9c29 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java @@ -22,69 +22,43 @@ import io.debezium.engine.DebeziumEngine; import io.debezium.engine.RecordChangeEvent; import io.pixelsdb.pixels.common.metadata.SchemaTableName; -import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; -import io.pixelsdb.pixels.sink.event.deserializer.TransactionStructMessageDeserializer; -import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.processor.StoppableProcessor; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; import io.pixelsdb.pixels.sink.provider.TransactionEventEngineProvider; -import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; -import io.pixelsdb.pixels.sink.sink.ProtoWriter; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; -import java.io.IOException; import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; /** - * @package: io.pixelsdb.pixels.sink + * @package: io.pixelsdb.pixels.source * @className: PixelsDebeziumConsumer * @author: AntiO2 * @date: 2025/9/25 12:51 */ public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer>, StoppableProcessor { - private final BlockingQueue rawTransactionQueue = new LinkedBlockingQueue<>(10000); private final String checkTransactionTopic; - private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; + private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; private final TableProviderAndProcessorPipelineManager tableProvidersManagerImpl = new TableProviderAndProcessorPipelineManager<>(); private final TransactionProcessor processor = new TransactionProcessor(transactionEventProvider); - private final Thread adapterThread; + private final Thread transactionProviderThread; + private final Thread transactionProcessorThread; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final PixelsSinkMode pixelsSinkMode; - private final ProtoWriter protoWriter; PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); public PixelsDebeziumConsumer() { this.checkTransactionTopic = pixelsSinkConfig.getDebeziumTopicPrefix() + ".transaction"; - adapterThread = new Thread(this.transactionEventProvider, "transaction-adapter"); - adapterThread.start(); - Thread processorThread = new Thread(processor, "debezium-processor"); - processorThread.start(); - pixelsSinkMode = pixelsSinkConfig.getPixelsSinkMode(); + this.transactionProviderThread = new Thread(this.transactionEventProvider, "transaction-adapter"); + this.transactionProcessorThread = new Thread(this.processor, "transaction-processor"); - if (pixelsSinkMode == PixelsSinkMode.PROTO) - { - try - { - this.protoWriter = new ProtoWriter(); - } catch (IOException e) - { - throw new RuntimeException(e); - } - } else - { - this.protoWriter = null; - } + this.transactionProcessorThread.start(); + this.transactionProviderThread.start(); } @@ -104,49 +78,10 @@ public void handleBatch(List> event, metricsFacade.recordDebeziumEvent(); if (isTransactionEvent(sourceRecord)) { - switch (pixelsSinkMode) - { - case RETINA -> - { - handleTransactionSourceRecord(sourceRecord); - } - case PROTO -> - { - metricsFacade.recordTransaction(); - SinkProto.TransactionMetadata transactionMetadata = TransactionStructMessageDeserializer.convertToTransactionMetadata(sourceRecord); - protoWriter.writeTrans(transactionMetadata); - } - default -> - { - throw new RuntimeException("Sink Mode " + pixelsSinkMode.toString() + "is not supported"); - } - } + handleTransactionSourceRecord(sourceRecord); } else { - switch (pixelsSinkMode) - { - case RETINA -> - { - handleRowChangeSourceRecord(sourceRecord); - } - case PROTO -> - { - try - { - RowChangeEvent rowChangeEvent = RowChangeEventStructDeserializer.convertToRowChangeEvent(sourceRecord); - protoWriter.writeRow(rowChangeEvent); - metricsFacade.recordRowEvent(); - } catch (SinkException e) - { - throw new RuntimeException(e); - } - - } - default -> - { - throw new RuntimeException("Sink Mode " + pixelsSinkMode.toString() + "is not supported"); - } - } + handleRowChangeSourceRecord(sourceRecord); } } finally { @@ -159,7 +94,7 @@ public void handleBatch(List> event, private void handleTransactionSourceRecord(SourceRecord sourceRecord) throws InterruptedException { - rawTransactionQueue.put(sourceRecord); + transactionEventProvider.putTransRawEvent(sourceRecord); } private void handleRowChangeSourceRecord(SourceRecord sourceRecord) @@ -180,17 +115,8 @@ private boolean isTransactionEvent(SourceRecord sourceRecord) @Override public void stopProcessor() { - adapterThread.interrupt(); + transactionProviderThread.interrupt(); processor.stopProcessor(); - if (protoWriter != null) - { - try - { - protoWriter.close(); - } catch (IOException e) - { - throw new RuntimeException(e); - } - } + transactionProcessorThread.interrupt(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java index 33cbb6a..1aa3bec 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java @@ -27,12 +27,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -/** - * @package: PACKAGE_NAME - * @className: io.pixelsdb.pixels.sink.source.SinkEngineSource - * @author: AntiO2 - * @date: 2025/9/25 09:17 - */ public class SinkEngineSource implements SinkSource { private final PixelsDebeziumConsumer consumer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java index 16a736b..6d8e010 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java @@ -21,7 +21,7 @@ import io.pixelsdb.pixels.sink.processor.StoppableProcessor; /** - * @package: io.pixelsdb.pixels.sink.processor + * @package: io.pixelsdb.pixels.sink.source * @className: SinkSource * @author: AntiO2 * @date: 2025/9/26 13:45 diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java similarity index 99% rename from src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java index 43681f6..1a16f2c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/CsvWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java @@ -14,7 +14,7 @@ * limitations under the License. * */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java similarity index 76% rename from src/main/java/io/pixelsdb/pixels/sink/sink/NoneWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index d4551c5..1514c2d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -15,15 +15,19 @@ * */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.util.MetricsFacade; import java.io.IOException; public class NoneWriter implements PixelsSinkWriter { + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + + @Override public void flush() { @@ -33,12 +37,17 @@ public void flush() @Override public boolean writeRow(RowChangeEvent rowChangeEvent) { + metricsFacade.recordRowEvent(); return true; } @Override public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) + { + metricsFacade.recordTransaction(); + } return true; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java similarity index 93% rename from src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java index d6e34d3..8e6af20 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; public enum PixelsSinkMode @@ -34,6 +34,6 @@ public static PixelsSinkMode fromValue(String value) return mode; } } - throw new RuntimeException(String.format("Can't convert %s to sink type", value)); + throw new RuntimeException(String.format("Can't convert %s to writer type", value)); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java similarity index 95% rename from src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java index f836115..744b46e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java similarity index 95% rename from src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java index 2423be8..bd44c34 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java @@ -15,11 +15,11 @@ * */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.sink.retina.RetinaWriter; +import io.pixelsdb.pixels.sink.writer.retina.RetinaWriter; import java.io.IOException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java index 0dbfd5b..21bbbf8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.common.physical.PhysicalWriter; @@ -34,7 +34,7 @@ import java.nio.ByteOrder; /** - * @package: io.pixelsdb.pixels.sink.sink + * @package: io.pixelsdb.pixels.sink.writer * @className: ProtoWriter * @author: AntiO2 * @date: 2025/10/5 07:10 diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java similarity index 96% rename from src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java index 7705d09..a873892 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/RotatingWriterManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.common.physical.PhysicalWriter; @@ -29,7 +29,7 @@ import java.io.IOException; /** - * @package: io.pixelsdb.pixels.sink.sink + * @package: io.pixelsdb.pixels.sink.writer * @className: RotatingWriterManager * @author: AntiO2 * @date: 2025/10/5 07:34 diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaServiceProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java similarity index 96% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaServiceProxy.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java index 5f01229..de26009 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaServiceProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java @@ -15,15 +15,15 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.RetinaException; import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.sink.PixelsSinkMode; import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.pixelsdb.pixels.sink.writer.PixelsSinkMode; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -117,7 +117,7 @@ public static RetinaWriteMode fromValue(String value) return mode; } } - throw new RuntimeException(String.format("Can't convert %s to sink type", value)); + throw new RuntimeException(String.format("Can't convert %s to Retina writer type", value)); } } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java similarity index 99% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 9159363..7f56f8d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransService; @@ -24,8 +24,8 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.sink.PixelsSinkWriter; import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import io.prometheus.client.Summary; import org.apache.commons.lang3.RandomUtils; import org.slf4j.Logger; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java similarity index 99% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContext.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 61d310a..05d5c31 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 2f4e3ce..2a01f39 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableCrossTxWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index aad0be2..b750e8a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.retina.RetinaProto; @@ -33,7 +33,7 @@ import java.util.concurrent.locks.ReentrantLock; /** - * @package: io.pixelsdb.pixels.sink.sink + * @package: io.pixelsdb.pixels.sink.writer.retina * @className: TableCrossTxWriter * @author: AntiO2 * @date: 2025/9/27 09:36 diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableSingleTxWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java index 3516871..49c3cc6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 3da1a47..94b877a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.retina.RetinaProto; @@ -37,7 +37,7 @@ import java.util.concurrent.locks.ReentrantLock; /** - * @package: io.pixelsdb.pixels.sink.sink + * @package: io.pixelsdb.pixels.sink.writer.retina * @className: TableWriter * @author: AntiO2 * @date: 2025/9/27 09:58 diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java similarity index 97% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java index a347d7a..6d6a4fd 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TableWriterProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java similarity index 95% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionMode.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java index 788b0ad..6d9d082 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; public enum TransactionMode { diff --git a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java similarity index 99% rename from src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionProxy.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index b11b60f..8b74456 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/sink/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -15,7 +15,7 @@ * */ -package io.pixelsdb.pixels.sink.sink.retina; +package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties index de99b69..6114330 100644 --- a/src/main/resources/pixels-sink.aws.properties +++ b/src/main/resources/pixels-sink.aws.properties @@ -7,7 +7,7 @@ bootstrap.servers=realtime-kafka-2:29092 group.id=3078 auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer -#value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer +#value.deserializer=io.pixelsdb.pixels.writer.deserializer.RowChangeEventAvroDeserializer value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config topic.prefix=postgresql.oltp_server @@ -19,14 +19,14 @@ sink.csv.enable_header=false sink.retina.embedded=false # stub or stream sink.retina.mode=stream -#sink.retina.mode=stub +#writer.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 sink.flush.interval.ms=100 sink.flush.batch.size=500 sink.max.retries=3 -## sink commit +## writer commit sink.commit.batch.size=500 sink.commit.batch.worker=32 sink.commit.batch.delay=200 @@ -38,7 +38,7 @@ sink.proto.maxRecords=1000000 sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction -#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +#transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=200 ## Batch or trans or record diff --git a/src/main/resources/pixels-sink.properties b/src/main/resources/pixels-sink.properties index d561fb6..41af3ec 100644 --- a/src/main/resources/pixels-sink.properties +++ b/src/main/resources/pixels-sink.properties @@ -4,7 +4,7 @@ group.id=docker-pixels-table auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventAvroDeserializer -#value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventJsonDeserializer +#value.deserializer=io.pixelsdb.pixels.writer.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config topic.prefix=oltp_server consumer.capture_database=pixels_realtime_crud @@ -24,7 +24,7 @@ sink.registry.url=http://apicurio:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionAvroMessageDeserializer -#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionJsonMessageDeserializer +#transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 ## batch or record sink.trans.mode=batch diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java index 4cc712a..74b9e4f 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/RetinaWriterTest.java @@ -22,8 +22,7 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.sink.retina.RetinaWriter; -import io.pixelsdb.pixels.sink.sink.retina.SinkContext; +import io.pixelsdb.pixels.sink.writer.retina.RetinaWriter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java index 7eb958c..783bbcb 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventDeserializerTest.java @@ -18,7 +18,6 @@ package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.junit.jupiter.api.Test; diff --git a/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java index 4150ebb..bdd14e4 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializerTest.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import io.pixelsdb.pixels.core.TypeDescription; -import io.pixelsdb.pixels.sink.event.deserializer.SchemaDeserializer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java similarity index 92% rename from src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java rename to src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java index cc2fd89..5fa8247 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestProtoWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java @@ -16,7 +16,7 @@ */ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import com.google.protobuf.ByteString; @@ -34,7 +34,7 @@ import java.nio.charset.StandardCharsets; /** - * @package: io.pixelsdb.pixels.sink.sink + * @package: io.pixelsdb.pixels.sink.writer * @className: TestProtoWriter * @author: AntiO2 * @date: 2025/10/5 09:24 @@ -47,8 +47,8 @@ public class TestProtoWriter @BeforeAll public static void setUp() throws IOException { - PixelsSinkConfigFactory.initialize("/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties"); -// PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties"); + PixelsSinkConfigFactory.initialize("/home/pixels/projects/pixels-writer/src/main/resources/pixels-writer.local.properties"); +// PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-writer/src/main/resources/pixels-writer.aws.properties"); } private static SinkProto.RowRecord getRowRecord(int i) @@ -107,7 +107,7 @@ public void testWriteTransInfo() @Test public void testWriteFile() throws IOException { - String path = "/home/pixels/projects/pixels-sink/tmp/write.dat"; + String path = "/home/pixels/projects/pixels-writer/tmp/write.dat"; PhysicalWriter writer = PhysicalWriterUtil.newPhysicalWriter(Storage.Scheme.file, path); int writeNum = 3; @@ -124,7 +124,7 @@ public void testWriteFile() throws IOException @Test public void testReadFile() throws IOException { - String path = "/home/pixels/projects/pixels-sink/tmp/write.dat"; + String path = "/home/pixels/projects/pixels-writer/tmp/write.dat"; PhysicalLocalReader reader = (PhysicalLocalReader) PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.file, path); int writeNum = 12; @@ -137,7 +137,7 @@ public void testReadFile() throws IOException @Test public void testReadEmptyFile() throws IOException { - String path = "/home/pixels/projects/pixels-sink/tmp/empty.dat"; + String path = "/home/pixels/projects/pixels-writer/tmp/empty.dat"; PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.file, path); int v = reader.readInt(ByteOrder.BIG_ENDIAN); diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java similarity index 99% rename from src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java rename to src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java index 0cc902c..fbd9962 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java @@ -1,4 +1,4 @@ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import com.google.protobuf.ByteString; import io.pixelsdb.pixels.common.exception.RetinaException; @@ -13,9 +13,9 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; -import io.pixelsdb.pixels.sink.sink.retina.RetinaServiceProxy; -import io.pixelsdb.pixels.sink.sink.retina.TransactionProxy; import io.pixelsdb.pixels.sink.util.DateUtil; +import io.pixelsdb.pixels.sink.writer.retina.RetinaServiceProxy; +import io.pixelsdb.pixels.sink.writer.retina.TransactionProxy; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; diff --git a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java similarity index 99% rename from src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java rename to src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java index 6cfd446..c43d220 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/sink/TpcHTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java @@ -1,4 +1,4 @@ -package io.pixelsdb.pixels.sink.sink; +package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.common.exception.RetinaException; import io.pixelsdb.pixels.common.exception.TransException; From 2681b1cbd543a95b8c0ca6bf0441e2c50ca63d62 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sat, 11 Oct 2025 13:48:16 +0000 Subject: [PATCH 08/53] Fix: Provider doesn't start --- .../pixels/sink/provider/EventProvider.java | 13 ++++++++++--- .../pixels/sink/provider/TableEventProvider.java | 14 ++++++++++++-- .../TableProviderAndProcessorPipelineManager.java | 3 ++- .../sink/provider/TransactionEventProvider.java | 5 +++++ .../provider/TransactionEventStorageProvider.java | 3 --- .../sink/source/AbstractSinkStorageSource.java | 2 +- src/main/resources/pixels-sink.aws.properties | 4 ++-- 7 files changed, 32 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index c3f63ec..de61e57 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -39,15 +39,20 @@ public abstract class EventProvider implements private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); + private Thread providerThread; + + @Override public void run() { - processLoop(); + providerThread = new Thread(this::processLoop); + providerThread.start(); } @Override public void close() { + this.providerThread.interrupt(); decodeExecutor.shutdown(); } @@ -98,7 +103,7 @@ protected void processLoop() TARGET_RECORD_T event = future.get(); if (event != null) { - metricsFacade.recordSerdRowChange(); + recordSerdEvent(); putTargetEvent(event); } } catch (ExecutionException e) @@ -139,7 +144,7 @@ protected void putTargetEvent(TARGET_RECORD_T event) } } - void putRawEvent(SOURCE_RECORD_T record) + protected void putRawEvent(SOURCE_RECORD_T record) { try { @@ -172,4 +177,6 @@ protected SOURCE_RECORD_T pollRawEvent(long remainingMs) return null; } } + + abstract protected void recordSerdEvent(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java index 4600d93..fc4e48a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java @@ -28,13 +28,23 @@ */ public abstract class TableEventProvider extends EventProvider { + protected void putRowChangeEvent(RowChangeEvent rowChangeEvent) + { + putTargetEvent(rowChangeEvent); + } + public RowChangeEvent getRowChangeEvent() { return getTargetEvent(); } - protected void putRowChangeEvent(RowChangeEvent rowChangeEvent) + protected void putRawRowChangeEvent(SOURCE_RECORD_T record) { - putTargetEvent(rowChangeEvent); + putRawEvent(record); + } + + final protected void recordSerdEvent() + { + metricsFacade.recordSerdRowChange(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index 1e10c52..304c82d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -56,6 +56,7 @@ public void routeRecord(Integer tableId, SOURCE_RECORD_T record) new TableProcessor(newPipeline) ); tableProcessor.run(); + newPipeline.run(); return newPipeline; }); pipeline.putRawEvent(record); @@ -67,7 +68,7 @@ private TableEventProvider createProvider(SOURCE_RECORD_T recor if (recordType == SourceRecord.class) { return new TableEventEngineProvider<>(); - } else if (recordType == ByteBuffer.class) + } else if (ByteBuffer.class.isAssignableFrom(recordType)) { return new TableEventStorageProvider<>(); } else diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java index 7de77c6..8072d53 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java @@ -30,4 +30,9 @@ public SinkProto.TransactionMetadata getTransaction() { return getTargetEvent(); } + + final protected void recordSerdEvent() + { + metricsFacade.recordSerdTxChange(); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java index daacd2b..4ecd555 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java @@ -28,16 +28,13 @@ public class TransactionEventStorageProvider extends TransactionEventProvider SinkProto.TransactionMetadata convertToTargetRecord(T record) { ByteBuffer buffer = (ByteBuffer) record; - try { SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(buffer); - metricsFacade.recordSerdTxChange(); return tx; } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } - } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 261e2c6..8fb1693 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -40,7 +40,7 @@ public abstract class AbstractSinkStorageSource implements SinkSource private final Map>> queueMap = new ConcurrentHashMap<>(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final TableProviderAndProcessorPipelineManager tablePipelineManager = new TableProviderAndProcessorPipelineManager(); - protected TransactionEventStorageProvider transactionEventProvider; + protected TransactionEventStorageProvider transactionEventProvider; protected TransactionProcessor transactionProcessor; protected Thread transactionProviderThread; protected Thread transactionProcessorThread; diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties index 6114330..8a221be 100644 --- a/src/main/resources/pixels-sink.aws.properties +++ b/src/main/resources/pixels-sink.aws.properties @@ -1,7 +1,7 @@ # engine | kafka | storage sink.datasource=storage # Sink Config: retina | csv | proto | none -sink.mode=retina +sink.mode=none # Kafka Config bootstrap.servers=realtime-kafka-2:29092 group.id=3078 @@ -47,7 +47,7 @@ sink.trans.mode=batch sink.monitor.enable=true sink.monitor.port=9464 sink.monitor.report.interval=5000 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/ParallelRocksDB2.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/readonly3.csv # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 From 411f7006a360def3a5c12394a034e371a0a911f6 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sun, 12 Oct 2025 22:12:59 +0800 Subject: [PATCH 09/53] hehe --- conf/pixels-sink.pg.properties | 79 +++++++++++++++++++ pixels-sink | 3 +- pom.xml | 2 +- src/main/resources/pixels-sink.aws.properties | 4 +- 4 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 conf/pixels-sink.pg.properties diff --git a/conf/pixels-sink.pg.properties b/conf/pixels-sink.pg.properties new file mode 100644 index 0000000..8dc99b5 --- /dev/null +++ b/conf/pixels-sink.pg.properties @@ -0,0 +1,79 @@ +# engine | kafka | storage +sink.datasource=engine +# Sink Config: retina | csv | proto | none +sink.mode=none +# Kafka Config +bootstrap.servers=localhost:29092 +group.id=3078 +auto.offset.reset=earliest +key.deserializer=org.apache.kafka.common.serialization.StringDeserializer +#value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer +value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer +# Topic & Database Config +topic.prefix=postgresql.oltp_server +consumer.capture_database=pixels_bench_sf1x +consumer.include_tables= +sink.csv.path=./data +sink.csv.enable_header=false +## Retina Config +sink.retina.embedded=false +# stub or stream +sink.retina.mode=stream +#sink.retina.mode=stub +sink.remote.host=localhost +sink.remote.port=29422 +sink.timeout.ms=5000 +sink.flush.interval.ms=100 +sink.flush.batch.size=100 +sink.max.retries=3 +## sink commit +sink.commit.batch.size=500 +sink.commit.batch.worker=32 +sink.commit.batch.delay=200 +## Proto Config +sink.proto.dir=file:///home/antio2/projects/pixels-sink/tmp +sink.proto.data=data +sink.proto.maxRecords=1000000 +## Schema Registry +sink.registry.url=http://localhost:8080/apis/registry/v2 +# Transaction Config +transaction.topic.suffix=transaction +#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer +sink.trans.batch.size=200 +## Batch or trans or record +sink.trans.mode=batch +# Sink Metrics +sink.monitor.enable=true +sink.monitor.port=9464 +sink.monitor.report.interval=5000 +sink.monitor.report.file=/home/antio2/projects/pixels-sink/tmp/pg.csv +# Interact with other rpc +sink.rpc.enable=true +sink.rpc.mock.delay=20 +# debezium +debezium.name=testEngine +debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector +debezium.provide.transaction.metadata=true +debezium.offset.storage=org.apache.kafka.connect.storage.FileOffsetBackingStore +debezium.offset.storage.file.filename=/tmp/offsets.dat +debezium.offset.flush.interval.ms=60000 +debezium.schema.history.internal=io.debezium.storage.file.history.FileSchemaHistory +debezium.schema.history.internal.file.filename=/tmp/schemahistory.dat +debezium.database.hostname=localhost +debezium.database.port=5432 +debezium.database.user=pixels +debezium.database.password=password +debezium.database.dbname=pixels_bench_sf1x +debezium.plugin.name=pgoutput +debezium.database.server.id=1 +debezium.schema.include.list=public +debezium.snapshot.mode=never +debezium.key.converter=org.apache.kafka.connect.json.JsonConverter +debezium.value.converter=org.apache.kafka.connect.json.JsonConverter +debezium.topic.prefix=postgresql.oltp_server +debezium.transforms=topicRouting +debezium.transforms.topicRouting.type=org.apache.kafka.connect.transforms.RegexRouter +debezium.transforms.topicRouting.regex=postgresql\\.oltp_server\\.public\\.(.*) +debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf1x.$1 + diff --git a/pixels-sink b/pixels-sink index cdd3269..bfe0ce1 100755 --- a/pixels-sink +++ b/pixels-sink @@ -9,8 +9,9 @@ SINK_DIR=$(dirname "$SOURCE_PATH") # export PIXELS_HOME="/home/ubuntu/opt/pixels" # Application properties file -PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" +#PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" #PROPERTIES_FILE="/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties" +PROPERTIES_FILE="${SINK_DIR}/conf/pixels-sink.pg.properties" # JVM config file JVM_CONFIG_FILE="${SINK_DIR}/conf/jvm.conf" diff --git a/pom.xml b/pom.xml index 37e269d..886e992 100644 --- a/pom.xml +++ b/pom.xml @@ -36,7 +36,7 @@ 0.9.0 3.8.0 5.8 - 1.18.36 + 1.18.42 3.2.3.Final 1.4.13 diff --git a/src/main/resources/pixels-sink.aws.properties b/src/main/resources/pixels-sink.aws.properties index 6114330..0571595 100644 --- a/src/main/resources/pixels-sink.aws.properties +++ b/src/main/resources/pixels-sink.aws.properties @@ -19,7 +19,7 @@ sink.csv.enable_header=false sink.retina.embedded=false # stub or stream sink.retina.mode=stream -#writer.retina.mode=stub +#sink.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 @@ -38,7 +38,7 @@ sink.proto.maxRecords=1000000 sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction -#transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer +#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=200 ## Batch or trans or record From b383177405b4a8aa8211e88e585868091f2d9ad2 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Mon, 13 Oct 2025 16:47:19 +0800 Subject: [PATCH 10/53] fix small bug --- conf/pixels-sink.pg.properties | 5 +- perf_web_monitor.py | 3 +- .../pixels/sink/config/PixelsSinkConfig.java | 4 + .../pixels/sink/provider/EventProvider.java | 2 +- .../provider/TableEventStorageProvider.java | 5 +- .../TransactionEventEngineProvider.java | 3 - .../source/AbstractSinkStorageSource.java | 108 +++++++++--------- .../sink/writer/PixelsSinkWriterFactory.java | 4 + 8 files changed, 71 insertions(+), 63 deletions(-) diff --git a/conf/pixels-sink.pg.properties b/conf/pixels-sink.pg.properties index 8dc99b5..136ca46 100644 --- a/conf/pixels-sink.pg.properties +++ b/conf/pixels-sink.pg.properties @@ -1,7 +1,8 @@ # engine | kafka | storage -sink.datasource=engine +sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=none +sink.storage.loop=true # Kafka Config bootstrap.servers=localhost:29092 group.id=3078 @@ -47,7 +48,7 @@ sink.trans.mode=batch sink.monitor.enable=true sink.monitor.port=9464 sink.monitor.report.interval=5000 -sink.monitor.report.file=/home/antio2/projects/pixels-sink/tmp/pg.csv +sink.monitor.report.file=/home/antio2/projects/pixels-sink/tmp/proto.csv # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 diff --git a/perf_web_monitor.py b/perf_web_monitor.py index 3d4675c..22447d4 100644 --- a/perf_web_monitor.py +++ b/perf_web_monitor.py @@ -5,7 +5,8 @@ from time import time # Configuration -DATA_DIR = "/home/ubuntu/pixels-sink/tmp" +# DATA_DIR = "/home/ubuntu/pixels-sink/tmp" +DATA_DIR = "/home/antio2/projects/pixels-sink/tmp" PORT = 8083 CACHE_TTL = 5 # seconds diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 57eaf5a..80fa20d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -150,6 +150,10 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.proto.maxRecords", defaultValue = PixelsSinkDefaultConfig.MAX_RECORDS_PER_FILE) private int maxRecordsPerFile; + @ConfigKey(value = "sink.storage.loop", defaultValue = "false") + private boolean sinkStorageLoop; + + public PixelsSinkConfig(String configFilePath) throws IOException { this.config = ConfigFactory.Instance(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index de61e57..82a6610 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -108,7 +108,7 @@ protected void processLoop() } } catch (ExecutionException e) { - LOGGER.warn("Decode failed: " + e.getCause()); + LOGGER.warn("Decode failed: {}", String.valueOf(e.getCause())); } } } catch (InterruptedException e) diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java index 4d912c5..f4d0af1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java @@ -51,10 +51,7 @@ RowChangeEvent convertToTargetRecord(T record) { SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); - } catch (InvalidProtocolBufferException e) - { - throw new RuntimeException(e); - } catch (SinkException e) + } catch (InvalidProtocolBufferException | SinkException e) { LOGGER.warning(e.getMessage()); return null; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java index d440b6d..f135342 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java @@ -45,9 +45,6 @@ SinkProto.TransactionMetadata convertToTargetRecord(T record) { SourceRecord sourceRecord = (SourceRecord) record; Struct value = (Struct) sourceRecord.value(); - metricsFacade.recordSerdTxChange(); return TransactionStructMessageDeserializer.convertToTransactionMetadata(value); } - - } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 8fb1693..2baa8ef 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -44,6 +44,7 @@ public abstract class AbstractSinkStorageSource implements SinkSource protected TransactionProcessor transactionProcessor; protected Thread transactionProviderThread; protected Thread transactionProcessorThread; + private final boolean storageLoopEnabled; protected AbstractSinkStorageSource() { @@ -52,6 +53,7 @@ protected AbstractSinkStorageSource() this.baseDir = pixelsSinkConfig.getSinkProtoDir(); this.etcdFileRegistry = new EtcdFileRegistry(topic, baseDir); this.files = this.etcdFileRegistry.listAllFiles(); + this.storageLoopEnabled = pixelsSinkConfig.isSinkStorageLoop(); this.transactionEventProvider = new TransactionEventStorageProvider<>(); this.transactionProviderThread = new Thread(transactionEventProvider); @@ -70,69 +72,71 @@ protected void handleTransactionSourceRecord(ByteBuffer record) @Override public void start() { + this.running.set(true); this.transactionProcessorThread.start(); this.transactionProviderThread.start(); - for (String file : files) - { - Storage.Scheme scheme = Storage.Scheme.fromPath(file); - LOGGER.info("Start read from file {}", file); - try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) + do { + for (String file : files) { - long offset = 0; - BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); - BlockingQueue> transQueue = new LinkedBlockingQueue<>(); - while (true) + Storage.Scheme scheme = Storage.Scheme.fromPath(file); + LOGGER.info("Start read from file {}", file); + try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) { - try + long offset = 0; + BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); + BlockingQueue> transQueue = new LinkedBlockingQueue<>(); + while (true) { - int key, valueLen; - reader.seek(offset); try { - key = reader.readInt(ByteOrder.BIG_ENDIAN); - valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) + int key, valueLen; + reader.seek(offset); + try + { + key = reader.readInt(ByteOrder.BIG_ENDIAN); + valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + } catch (IOException e) + { + // EOF + break; + } + + ProtoType protoType = getProtoType(key); + offset += Integer.BYTES * 2; + CompletableFuture valueFuture = reader.readAsync(offset, valueLen) + .thenApply(this::copyToHeap) + .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); + // move offset for next record + offset += valueLen; + + + // Get or create queue + BlockingQueue> queue = + queueMap.computeIfAbsent(key, + k -> new LinkedBlockingQueue<>(maxQueueCapacity)); + + // Put future in queue + queue.put(valueFuture); + + // Start consumer thread if not exists + consumerThreads.computeIfAbsent(key, k -> + { + Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); + t.setName("consumer-" + key); + t.start(); + return t; + }); + } catch (IOException | InterruptedException e) { - // EOF break; } - - ProtoType protoType = getProtoType(key); - offset += Integer.BYTES * 2; - CompletableFuture valueFuture = reader.readAsync(offset, valueLen) - .thenApply(this::copyToHeap) - .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); - // move offset for next record - offset += valueLen; - - - // Get or create queue - BlockingQueue> queue = - queueMap.computeIfAbsent(key, - k -> new LinkedBlockingQueue<>(maxQueueCapacity)); - - // Put future in queue - queue.put(valueFuture); - - // Start consumer thread if not exists - consumerThreads.computeIfAbsent(key, k -> - { - Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); - t.setName("consumer-" + key); - t.start(); - return t; - }); - } catch (IOException | InterruptedException e) - { - break; } + } catch (IOException e) + { + throw new RuntimeException(e); } - } catch (IOException e) - { - throw new RuntimeException(e); } - - } + } while (storageLoopEnabled && isRunning()); // signal all queues to stop queueMap.values().forEach(q -> @@ -201,16 +205,16 @@ private void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer) tablePipelineManager.routeRecord(key, dataBuffer); } - @Override public boolean isRunning() { - return false; + return running.get(); } @Override public void stopProcessor() { + running.set(false); transactionProviderThread.interrupt(); transactionProcessorThread.interrupt(); transactionProcessor.stopProcessor(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java index bd44c34..2801cef 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java @@ -44,12 +44,16 @@ static public PixelsSinkWriter getWriter() { case CSV: writer = new CsvWriter(); + break; case RETINA: writer = new RetinaWriter(); + break; case PROTO: writer = new ProtoWriter(); + break; case NONE: writer = new NoneWriter(); + break; } } catch (IOException e) { From e1a76da05a82c7bc749de21f99e5f4e5c891e0a6 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 14 Oct 2025 09:57:39 +0000 Subject: [PATCH 11/53] Feat: add bucket --- conf/pixels-sink.aws.properties | 80 +++++++++++++++++++ perf_web_monitor.py | 4 +- pixels-sink | 3 +- .../pixels/sink/config/PixelsSinkConfig.java | 3 + .../pixels/sink/event/RowChangeEvent.java | 57 ++++++++++++- .../writer/retina/SinkContextManager.java | 7 +- .../writer/retina/TableCrossTxWriter.java | 12 +-- .../sink/writer/retina/TableWriterProxy.java | 14 ++-- src/main/resources/log4j2.properties | 12 +++ .../pixels/sink/event/RowChangeEventTest.java | 43 ++++++++++ .../writer/retina/TableWriterProxyTest.java | 35 ++++++++ src/test/resources/log4j2.properties | 2 +- 12 files changed, 256 insertions(+), 16 deletions(-) create mode 100644 conf/pixels-sink.aws.properties create mode 100644 src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties new file mode 100644 index 0000000..435e583 --- /dev/null +++ b/conf/pixels-sink.aws.properties @@ -0,0 +1,80 @@ +# engine | kafka | storage +sink.datasource=storage +# Sink Config: retina | csv | proto | none +sink.mode=retina +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/memory_index.csv +sink.storage.loop=true +# Kafka Config +bootstrap.servers=realtime-kafka-2:29092 +group.id=3078 +auto.offset.reset=earliest +key.deserializer=org.apache.kafka.common.serialization.StringDeserializer +#value.deserializer=io.pixelsdb.pixels.writer.deserializer.RowChangeEventAvroDeserializer +value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer +# Topic & Database Config +topic.prefix=postgresql.oltp_server +consumer.capture_database=pixels_bench_sf1x +consumer.include_tables= +sink.csv.path=./data +sink.csv.enable_header=false +## Retina Config +sink.retina.embedded=false +# stub or stream +sink.retina.mode=stream +sink.retina.bucket=16 +#writer.retina.mode=stub +sink.remote.host=localhost +sink.remote.port=29422 +sink.timeout.ms=5000 +sink.flush.interval.ms=2000 +sink.flush.batch.size=500 +sink.max.retries=3 +## writer commit +sink.commit.batch.size=500 +sink.commit.batch.worker=32 +sink.commit.batch.delay=200 +## Proto Config +sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp +sink.proto.data=data +sink.proto.maxRecords=1000000 +## Schema Registry +sink.registry.url=http://localhost:8080/apis/registry/v2 +# Transaction Config +transaction.topic.suffix=transaction +#transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer +transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer +sink.trans.batch.size=200 +## Batch or trans or record +sink.trans.mode=batch +# Sink Metrics +sink.monitor.enable=true +sink.monitor.port=9464 +sink.monitor.report.interval=5000 +# Interact with other rpc +sink.rpc.enable=true +sink.rpc.mock.delay=20 +# debezium +debezium.name=testEngine +debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector +debezium.provide.transaction.metadata=true +debezium.offset.storage=org.apache.kafka.connect.storage.FileOffsetBackingStore +debezium.offset.storage.file.filename=/tmp/offsets.dat +debezium.offset.flush.interval.ms=60000 +debezium.schema.history.internal=io.debezium.storage.file.history.FileSchemaHistory +debezium.schema.history.internal.file.filename=/tmp/schemahistory.dat +debezium.database.hostname=realtime-pg-2 +debezium.database.port=5432 +debezium.database.user=pixels +debezium.database.password=pixels_realtime_crud +debezium.database.dbname=pixels_bench_sf1x +debezium.plugin.name=pgoutput +debezium.database.server.id=1 +debezium.schema.include.list=public +debezium.snapshot.mode=never +debezium.key.converter=org.apache.kafka.connect.json.JsonConverter +debezium.value.converter=org.apache.kafka.connect.json.JsonConverter +debezium.topic.prefix=postgresql.oltp_server +debezium.transforms=topicRouting +debezium.transforms.topicRouting.type=org.apache.kafka.connect.transforms.RegexRouter +debezium.transforms.topicRouting.regex=postgresql\\.oltp_server\\.public\\.(.*) +debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf1x.$1 diff --git a/perf_web_monitor.py b/perf_web_monitor.py index 22447d4..b7cbc29 100644 --- a/perf_web_monitor.py +++ b/perf_web_monitor.py @@ -5,8 +5,8 @@ from time import time # Configuration -# DATA_DIR = "/home/ubuntu/pixels-sink/tmp" -DATA_DIR = "/home/antio2/projects/pixels-sink/tmp" +DATA_DIR = "/home/ubuntu/pixels-sink/tmp" +# DATA_DIR = "/home/antio2/projects/pixels-sink/tmp" PORT = 8083 CACHE_TTL = 5 # seconds diff --git a/pixels-sink b/pixels-sink index bfe0ce1..fed1eba 100755 --- a/pixels-sink +++ b/pixels-sink @@ -11,7 +11,8 @@ SINK_DIR=$(dirname "$SOURCE_PATH") # Application properties file #PROPERTIES_FILE="/home/ubuntu/pixels-sink/src/main/resources/pixels-sink.aws.properties" #PROPERTIES_FILE="/home/pixels/projects/pixels-sink/src/main/resources/pixels-sink.local.properties" -PROPERTIES_FILE="${SINK_DIR}/conf/pixels-sink.pg.properties" +#PROPERTIES_FILE="${SINK_DIR}/conf/pixels-sink.pg.properties" +PROPERTIES_FILE="${SINK_DIR}/conf/pixels-sink.aws.properties" # JVM config file JVM_CONFIG_FILE="${SINK_DIR}/conf/jvm.conf" diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 80fa20d..9cac6ca 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -40,6 +40,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.retina.mode", defaultValue = PixelsSinkDefaultConfig.SINK_RETINA_MODE) private RetinaServiceProxy.RetinaWriteMode retinaWriteMode; + @ConfigKey(value = "sink.retina.bucket", defaultValue = "1") + private int retinaBucketNum; + @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) private TransactionMode transactionMode; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 7ce2b06..b08b110 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -17,11 +17,14 @@ package io.pixelsdb.pixels.sink.event; +import com.google.common.hash.Hashing; import com.google.protobuf.ByteString; import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.index.IndexProto; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadata; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; @@ -29,6 +32,7 @@ import io.prometheus.client.Summary; import lombok.Getter; import lombok.Setter; +import org.apache.logging.log4j.core.util.Assert; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -63,10 +67,17 @@ public class RowChangeEvent @Getter private IndexProto.IndexKey afterKey; + private boolean indexKeyInited = false; + + @Getter + private final long tableId; + public RowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException { this.rowRecord = rowRecord; - this.schema = TableMetadataRegistry.Instance().getTypeDescription(getSchemaName(), getTable()); + TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + this.schema = tableMetadataRegistry.getTypeDescription(getSchemaName(), getTable()); + this.tableId = tableMetadataRegistry.getTableId(getSchemaName(), getTable()); initColumnValueMap(); initIndexKey(); } @@ -75,6 +86,9 @@ public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) thr { this.rowRecord = rowRecord; this.schema = schema; + + TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + this.tableId = tableMetadataRegistry.getTableId(getSchemaName(), getTable()); initColumnValueMap(); // initIndexKey(); } @@ -102,6 +116,11 @@ private void initColumnValueMap(SinkProto.RowValue rowValue, Map rowValue) { List keyColumnNames = tableMetadata.getKeyColumnNames(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 2a01f39..580e977 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -30,6 +30,8 @@ public class SinkContextManager { private static final Logger LOGGER = LoggerFactory.getLogger(SinkContextManager.class); + private static final Logger BUCKET_TRACE_LOGGER = LoggerFactory.getLogger("bucket_trace"); + private final static SinkContextManager INSTANCE = new SinkContextManager(); private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(); @@ -119,7 +121,10 @@ protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws String table = event.getTable(); event.setTimeStamp(ctx.getTimestamp()); event.initIndexKey(); - tableWriterProxy.getTableWriter(table).write(event, ctx); + int bucket = event.getBucketFromIndex(); + long tableId = event.getTableId(); + BUCKET_TRACE_LOGGER.info("{}\t{}\t{}", tableId, bucket, event.getAfterKey().getKey().asReadOnlyByteBuffer().getInt()); + tableWriterProxy.getTableWriter(table, tableId, bucket).write(event, ctx); } protected SinkContext getSinkContext(String txId) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index b750e8a..0a1645e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -44,11 +44,13 @@ public class TableCrossTxWriter extends TableWriter private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); private final int flushBatchSize; private final ReentrantLock writeLock = new ReentrantLock(); + private final int bucketId; - public TableCrossTxWriter(String t) + public TableCrossTxWriter(String t, int bucketId) { super(t); flushBatchSize = config.getFlushBatchSize(); + this.bucketId = bucketId; } /** @@ -89,7 +91,7 @@ public void flush() { if (smallBatch != null && !smallBatch.isEmpty()) { - tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch)); + tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucket(bucketId).build()); tableUpdateCount.add(smallBatch.size()); } txIds.add(currTxId); @@ -102,7 +104,7 @@ public void flush() if (smallBatch != null) { - tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch)); + tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucket(bucketId).build()); tableUpdateCount.add(smallBatch.size()); } @@ -131,7 +133,7 @@ private void updateCtxCounters(List txIds, List fullTableName, L } } - private RetinaProto.TableUpdateData buildTableUpdateDataFromBatch(String txId, List smallBatch) + private RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(txId); try @@ -165,7 +167,7 @@ private RetinaProto.TableUpdateData buildTableUpdateDataFromBatch(String txId, L { throw new RuntimeException("Flush failed for table " + tableName, e); } - return builder.build(); + return builder; } @Override diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java index 6d6a4fd..d0a7a1e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java @@ -28,7 +28,9 @@ public class TableWriterProxy private final static TableWriterProxy INSTANCE = new TableWriterProxy(); private final TransactionMode transactionMode; - private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); + + record TableKey(long tableId, int bucket) { } + private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); private TableWriterProxy() { @@ -41,19 +43,21 @@ protected static TableWriterProxy getInstance() return INSTANCE; } - protected TableWriter getTableWriter(String tableName) + protected TableWriter getTableWriter(String tableName, long tableId, int bucket) { - return WRITER_REGISTRY.computeIfAbsent(tableName, t -> + // warn: we assume table id is less than INT.MAX + TableKey key = new TableKey(tableId, bucket); + return WRITER_REGISTRY.computeIfAbsent(key, t -> { switch (transactionMode) { case SINGLE -> { - return new TableSingleTxWriter(t); + return new TableSingleTxWriter(tableName); } case BATCH -> { - return new TableCrossTxWriter(t); + return new TableCrossTxWriter(tableName, bucket); } default -> { diff --git a/src/main/resources/log4j2.properties b/src/main/resources/log4j2.properties index dcc5c79..f385702 100644 --- a/src/main/resources/log4j2.properties +++ b/src/main/resources/log4j2.properties @@ -29,3 +29,15 @@ log4j2.logger.io.grpc.netty.shaded.io.grpc.netty.NettyClientHandler=OFF logger.netty-shaded.name=io.grpc.netty.shaded.io.netty logger.netty-shaded.level=info logger.netty-shaded.additivity=false + +appender.bucket_trace.type=File +appender.bucket_trace.name=BUCKET_TRACE +appender.bucket_trace.append=true +appender.bucket_trace.fileName=${env:PIXELS_HOME}/logs/bucket_trace.log +appender.bucket_trace.layout.type=PatternLayout +appender.bucket_trace.layout.pattern=%m%n + +logger.bucket_trace.name=bucket_trace +logger.bucket_trace.level=warn +logger.bucket_trace.additivity=false +logger.bucket_trace.appenderRef.bucket_trace.ref=BUCKET_TRACE diff --git a/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java new file mode 100644 index 0000000..4ce0766 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java @@ -0,0 +1,43 @@ +package io.pixelsdb.pixels.sink.event; + +import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.index.IndexProto; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public class RowChangeEventTest +{ + private static Logger LOGGER = LoggerFactory.getLogger(RowChangeEventTest.class); + + @BeforeAll + public static void init() throws IOException + { + PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/conf/pixels-sink.aws.properties"); + } + + + @Test + public void testSameHash() + { + for(int i = 0; i < 10; ++i) + { + ByteString indexKey = getIndexKey(0); + int bucket = RowChangeEvent.getBucketIdFromByteBuffer(indexKey); + LOGGER.info("Bucket: {}", bucket); + } + } + + private ByteString getIndexKey(int key) + { + int keySize = Integer.BYTES; + ByteBuffer byteBuffer = ByteBuffer.allocate(keySize); + byteBuffer.putInt(key); + return ByteString.copyFrom(byteBuffer.rewind()); + } +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java b/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java new file mode 100644 index 0000000..af74333 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java @@ -0,0 +1,35 @@ +package io.pixelsdb.pixels.sink.writer.retina; + +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEventTest; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class TableWriterProxyTest +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TableWriterProxyTest.class); + + + String tableName = "test"; + + @BeforeAll + public static void init() throws IOException + { + PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/conf/pixels-sink.aws.properties"); + } + + @Test + public void testGetSameTableWriter() throws IOException + { + TableWriterProxy tableWriterProxy = TableWriterProxy.getInstance(); + + for(int i = 0; i < 10 ; i++) + { + TableWriter tableWriter = tableWriterProxy.getTableWriter(tableName, 0, 0); + } + } +} diff --git a/src/test/resources/log4j2.properties b/src/test/resources/log4j2.properties index 8d15751..09ab7d9 100644 --- a/src/test/resources/log4j2.properties +++ b/src/test/resources/log4j2.properties @@ -1,7 +1,7 @@ status=info name=pixels-sink filter.threshold.type=ThresholdFilter -filter.threshold.level=warn +filter.threshold.level=info appender.console.type=Console appender.console.name=STDOUT appender.console.layout.type=PatternLayout From 3911a79ccbc6c20e04b94c7e7e18040496813aec Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 15 Oct 2025 22:03:17 +0800 Subject: [PATCH 12/53] Retina Writer: Handle Pk ChangeEvent --- .../pixels/sink/event/RowChangeEvent.java | 29 +++++++-- .../sink/writer/retina/RetinaWriter.java | 2 +- .../sink/writer/retina/SinkContext.java | 25 -------- .../writer/retina/SinkContextManager.java | 61 +++++++++++++++++-- 4 files changed, 83 insertions(+), 34 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index b08b110..2632ddc 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -142,21 +142,42 @@ public void initIndexKey() throws SinkException indexKeyInited = true; } - public int getBucketFromIndex() + public int getBeforeBucketFromIndex() { assert indexKeyInited; - if(hasBeforeData()) { return getBucketFromIndexKey(beforeKey); } + throw new IllegalCallerException("Event dosen't have before data"); + } + + public boolean isPkChanged() throws SinkException + { + if(!indexKeyInited) + { + initIndexKey(); + } + if(getOp() != SinkProto.OperationType.UPDATE) + { + return false; + } + + ByteString beforeKey = getBeforeKey().getKey(); + ByteString afterKey = getAfterKey().getKey(); + + return !beforeKey.equals(afterKey); + } + + public int getAfterBucketFromIndex() + { + assert indexKeyInited; if(hasAfterData()) { return getBucketFromIndexKey(afterKey); } - - return 0; + throw new IllegalCallerException("Event dosen't have after data"); } protected static int getBucketFromIndexKey(IndexProto.IndexKey indexKey) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 7f56f8d..7f4be11 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -170,7 +170,7 @@ private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, S LOGGER.trace("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), txEnd.getDataCollectionsList().stream() .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCursors.getOrDefault(dc.getDataCollection(), 0L) + + ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + "/" + dc.getEventCount()) .collect(Collectors.joining(", "))); if (ctx == null) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 05d5c31..da21720 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -45,7 +45,6 @@ public class SinkContext final String sourceTxId; - final Map tableCursors = new ConcurrentHashMap<>(); final Map tableCounters = new ConcurrentHashMap<>(); final AtomicInteger pendingEvents = new AtomicInteger(0); final CompletableFuture completionFuture = new CompletableFuture<>(); @@ -67,20 +66,6 @@ public SinkContext(String sourceTxId, TransContext pixelsTransCtx) this.pixelsTransCtx = pixelsTransCtx; } - boolean isReadyForDispatch(String table, long collectionOrder) - { - lock.lock(); - boolean ready = tableCursors - .computeIfAbsent(table, k -> 1L) >= collectionOrder; - lock.unlock(); - return ready; - } - - void updateCursor(String table, long currentOrder) - { - tableCursors.compute(table, (k, v) -> - (v == null) ? currentOrder + 1 : Math.max(v, currentOrder + 1)); - } void updateCounter(String table) { @@ -106,11 +91,6 @@ public Condition getTableCounterCond() return tableCounterCond; } - Set getTrackedTables() - { - return tableCursors.keySet(); - } - boolean isCompleted(SinkProto.TransactionMetadata tx) { for (SinkProto.DataCollection dataCollection : tx.getDataCollectionsList()) @@ -161,11 +141,6 @@ public String getSourceTxId() return sourceTxId; } - public Map getTableCursors() - { - return tableCursors; - } - public Map getTableCounters() { return tableCounters; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 580e977..6e4dbbf 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -17,6 +17,8 @@ package io.pixelsdb.pixels.sink.writer.retina; +import io.pixelsdb.pixels.core.TypeDescription; +import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import org.slf4j.Logger; @@ -117,16 +119,67 @@ private void handleOrphanEvents(SinkContext ctx) throws SinkException } protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException + { + event.initIndexKey(); + switch (event.getOp()) + { + case UPDATE -> + { + if(!event.isPkChanged()) + { + writeBeforeEvent(ctx, event); + } else + { + TypeDescription typeDescription = event.getSchema(); + ctx.updateCounter(event.getFullTableName(), -1L); + + SinkProto.RowRecord.Builder deleteBuilder = event.getRowRecord().toBuilder() + .clearAfter().setOp(SinkProto.OperationType.DELETE); + RowChangeEvent deleteEvent = new RowChangeEvent(deleteBuilder.build(), typeDescription); + deleteEvent.initIndexKey(); + writeBeforeEvent(ctx, deleteEvent); + + SinkProto.RowRecord.Builder insertBuilder = event.getRowRecord().toBuilder() + .clearBefore().setOp(SinkProto.OperationType.INSERT); + RowChangeEvent insertEvent = new RowChangeEvent(insertBuilder.build(), typeDescription); + insertEvent.initIndexKey(); + writeAfterEvent(ctx, deleteEvent); + } + } + case DELETE -> + { + writeBeforeEvent(ctx, event); + } + case INSERT, SNAPSHOT -> + { + writeAfterEvent(ctx, event); + } + case UNRECOGNIZED -> + { + return; + } + } + } + + private boolean writeBeforeEvent(SinkContext ctx, RowChangeEvent event) + { + String table = event.getTable(); + event.setTimeStamp(ctx.getTimestamp()); + long tableId = event.getTableId(); + int beforeBucketFromIndex = event.getBeforeBucketFromIndex(); + return tableWriterProxy.getTableWriter(table, tableId, beforeBucketFromIndex).write(event, ctx); + } + + private boolean writeAfterEvent(SinkContext ctx, RowChangeEvent event) { String table = event.getTable(); event.setTimeStamp(ctx.getTimestamp()); - event.initIndexKey(); - int bucket = event.getBucketFromIndex(); long tableId = event.getTableId(); - BUCKET_TRACE_LOGGER.info("{}\t{}\t{}", tableId, bucket, event.getAfterKey().getKey().asReadOnlyByteBuffer().getInt()); - tableWriterProxy.getTableWriter(table, tableId, bucket).write(event, ctx); + int afterBucketFromIndex = event.getAfterBucketFromIndex(); + return tableWriterProxy.getTableWriter(table, tableId, afterBucketFromIndex).write(event, ctx); } + protected SinkContext getSinkContext(String txId) { return activeTxContexts.get(txId); From f25442a36442f1c74a4efe34c59442359973755b Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 16 Oct 2025 09:10:08 +0800 Subject: [PATCH 13/53] Handle Retina Service Error --- .../sink/writer/retina/RetinaWriter.java | 108 +----------------- .../sink/writer/retina/SinkContext.java | 79 +++---------- .../writer/retina/SinkContextManager.java | 82 +++++++++++++ .../writer/retina/TableCrossTxWriter.java | 21 +++- 4 files changed, 119 insertions(+), 171 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 7f4be11..d1022e1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -47,7 +47,7 @@ public class RetinaWriter implements PixelsSinkWriter private final ScheduledExecutorService timeoutScheduler = Executors.newSingleThreadScheduledExecutor(); private final TransactionProxy transactionProxy = TransactionProxy.Instance(); - private final TransService transService; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); @@ -55,7 +55,6 @@ public class RetinaWriter implements PixelsSinkWriter public RetinaWriter() { - transService = TransService.Instance(); this.sinkContextManager = SinkContextManager.getInstance(); } @@ -139,116 +138,13 @@ private void startTransSync(String sourceTxId) throws SinkException private void handleTxEnd(SinkProto.TransactionMetadata txEnd) { - String txId = txEnd.getId(); - SinkContext ctx = sinkContextManager.getSinkContext(txId); - transactionExecutor.submit(() -> { - processTxCommit(txEnd, txId, ctx); + sinkContextManager.processTxCommit(txEnd); } ); - switch (pixelsSinkConfig.getTransactionMode()) - { - -// case BATCH -> -// { -// processTxCommit(txEnd, txId, ctx); -// } -// case RECORD -> -// { -// transactionExecutor.submit(() -> -// { -// processTxCommit(txEnd, txId, ctx); -// } -// ); -// } - } } - private void processTxCommit(SinkProto.TransactionMetadata txEnd, String txId, SinkContext ctx) - { - LOGGER.trace("Begin to Commit transaction: {}, total event {}; Data Collection {}", txId, txEnd.getEventCount(), - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - if (ctx == null) - { - LOGGER.warn("Sink Context is null"); - return; - } - - try - { - try - { - ctx.tableCounterLock.lock(); - while (!ctx.isCompleted(txEnd)) - { - LOGGER.debug("TX End Get Lock {}", txId); - LOGGER.debug("Waiting for events in TX {}: {}", txId, - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - ctx.tableCounterCond.await(); - } - } finally - { - ctx.tableCounterLock.unlock(); - } - - - sinkContextManager.removeSinkContext(txId); - boolean res = true; - if (res) - { - LOGGER.trace("Committed transaction: {}", txId); - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - transactionProxy.commitTransAsync(ctx.getPixelsTransCtx()); - } else - { - LOGGER.info("Abort transaction: {}", txId); - Summary.Timer transLatencyTimer = metricsFacade.startTransLatencyTimer(); - CompletableFuture.runAsync(() -> - { - try - { - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); - } catch (TransException e) - { - throw new RuntimeException(e); - } - }).whenComplete((v, ex) -> - { - transLatencyTimer.close(); - if (ex != null) - { - LOGGER.error("Rollback failed", ex); - } - }); - } - } catch (InterruptedException e) - { - try - { - LOGGER.info("Catch Exception, Abort transaction: {}", txId); - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); - } catch (TransException ex) - { - LOGGER.error("Failed to abort transaction {}", txId); - ex.printStackTrace(); - LOGGER.error(ex.getMessage()); - throw new RuntimeException(ex); - } - LOGGER.error(e.getMessage()); - LOGGER.error("Failed to commit transaction {}", txId, e); - } - } - - private void handleNonTxEvent(RowChangeEvent event) throws SinkException { // virtual tx diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index da21720..5bb74cf 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -21,12 +21,13 @@ import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import lombok.Getter; +import lombok.Setter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Map; import java.util.Queue; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; @@ -37,22 +38,36 @@ public class SinkContext { private static final Logger LOGGER = LoggerFactory.getLogger(SinkContext.class); + @Getter final ReentrantLock lock = new ReentrantLock(); + @Getter final Condition cond = lock.newCondition(); // this cond is wait for pixels tx + @Getter final ReentrantLock tableCounterLock = new ReentrantLock(); + @Getter final Condition tableCounterCond = tableCounterLock.newCondition(); + @Getter final String sourceTxId; + @Getter final Map tableCounters = new ConcurrentHashMap<>(); + @Getter final AtomicInteger pendingEvents = new AtomicInteger(0); + @Getter final CompletableFuture completionFuture = new CompletableFuture<>(); + @Getter final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - volatile boolean completed = false; + @Getter Queue orphanEvent = new ConcurrentLinkedQueue<>(); + @Getter + @Setter private TransContext pixelsTransCtx; + @Setter + @Getter + private boolean failed = false; public SinkContext(String sourceTxId) { @@ -81,16 +96,6 @@ public void updateCounter(String table, long count) tableCounterLock.unlock(); } - public ReentrantLock getTableCounterLock() - { - return tableCounterLock; - } - - public Condition getTableCounterCond() - { - return tableCounterCond; - } - boolean isCompleted(SinkProto.TransactionMetadata tx) { for (SinkProto.DataCollection dataCollection : tx.getDataCollectionsList()) @@ -121,54 +126,4 @@ public void bufferOrphanedEvent(RowChangeEvent event) orphanEvent.add(event); } - public Queue getOrphanEvent() - { - return orphanEvent; - } - - public ReentrantLock getLock() - { - return lock; - } - - public Condition getCond() - { - return cond; - } - - public String getSourceTxId() - { - return sourceTxId; - } - - public Map getTableCounters() - { - return tableCounters; - } - - public AtomicInteger getPendingEvents() - { - return pendingEvents; - } - - public CompletableFuture getCompletionFuture() - { - return completionFuture; - } - - public TableMetadataRegistry getTableMetadataRegistry() - { - return tableMetadataRegistry; - } - - public TransContext getPixelsTransCtx() - { - return pixelsTransCtx; - } - - public void setPixelsTransCtx(TransContext pixelsTransCtx) - { - this.pixelsTransCtx = pixelsTransCtx; - } - } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 6e4dbbf..c9a1aa7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -17,6 +17,8 @@ package io.pixelsdb.pixels.sink.writer.retina; +import io.pixelsdb.pixels.common.exception.TransException; +import io.pixelsdb.pixels.common.transaction.TransService; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; @@ -25,9 +27,11 @@ import org.slf4j.LoggerFactory; import java.util.Queue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; public class SinkContextManager { @@ -39,6 +43,7 @@ public class SinkContextManager private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(); private final TransactionProxy transactionProxy = TransactionProxy.Instance(); + private final TransService transService = TransService.Instance(); private final TableWriterProxy tableWriterProxy; private SinkContextManager() @@ -105,6 +110,83 @@ protected void startTransSync(String sourceTxId) LOGGER.trace("Begin Tx Sync: {}", sourceTxId); } + void processTxCommit(SinkProto.TransactionMetadata txEnd) + { + String txId = txEnd.getId(); + SinkContext ctx = getSinkContext(txId); + if (ctx == null) + { + LOGGER.warn("Sink Context is null"); + return; + } + + try + { + try + { + ctx.tableCounterLock.lock(); + while (!ctx.isCompleted(txEnd)) + { + LOGGER.debug("TX End Get Lock {}", txId); + LOGGER.debug("Waiting for events in TX {}: {}", txId, + txEnd.getDataCollectionsList().stream() + .map(dc -> dc.getDataCollection() + "=" + + ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + + "/" + dc.getEventCount()) + .collect(Collectors.joining(", "))); + ctx.tableCounterCond.await(); + } + } finally + { + ctx.tableCounterLock.unlock(); + } + + + removeSinkContext(txId); + boolean failed = ctx.isFailed(); + if (!failed) + { + LOGGER.trace("Committed transaction: {}", txId); + transactionProxy.commitTransAsync(ctx.getPixelsTransCtx()); + } else + { + LOGGER.info("Abort transaction: {}", txId); + CompletableFuture.runAsync(() -> + { + try + { + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); + } catch (TransException e) + { + throw new RuntimeException(e); + } + }).whenComplete((v, ex) -> + { + if (ex != null) + { + LOGGER.error("Rollback failed", ex); + } + }); + } + } catch (InterruptedException e) + { + try + { + LOGGER.info("Catch Exception, Abort transaction: {}", txId); + transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); + } catch (TransException ex) + { + LOGGER.error("Failed to abort transaction {}", txId); + ex.printStackTrace(); + LOGGER.error(ex.getMessage()); + throw new RuntimeException(ex); + } + LOGGER.error(e.getMessage()); + LOGGER.error("Failed to commit transaction {}", txId, e); + } + } + + private void handleOrphanEvents(SinkContext ctx) throws SinkException { Queue buffered = ctx.getOrphanEvent(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 0a1645e..2057cfb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -91,7 +91,7 @@ public void flush() { if (smallBatch != null && !smallBatch.isEmpty()) { - tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucket(bucketId).build()); + tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); tableUpdateCount.add(smallBatch.size()); } txIds.add(currTxId); @@ -104,7 +104,7 @@ public void flush() if (smallBatch != null) { - tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucket(bucketId).build()); + tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); tableUpdateCount.add(smallBatch.size()); } @@ -113,7 +113,13 @@ public void flush() updateRecordResponseCompletableFuture.thenAccept( resp -> { - updateCtxCounters(txIds, fullTableName, tableUpdateCount); + if(resp.getHeader().getErrorCode() != 0) + { + failCtxs(txIds); + } else + { + updateCtxCounters(txIds, fullTableName, tableUpdateCount); + } } ); } finally @@ -122,6 +128,15 @@ public void flush() } } + private void failCtxs(List txIds) + { + for (String writeTxId : txIds) + { + SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); + sinkContext.setFailed(true); + } + } + private void updateCtxCounters(List txIds, List fullTableName, List tableUpdateCount) { for (int i = 0; i < txIds.size(); i++) From 85b1d02f7ebfe8172ee04569aaf9f5124488d642 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 16 Oct 2025 13:17:11 +0000 Subject: [PATCH 14/53] update retina rpc --- conf/pixels-sink.aws.properties | 4 +- .../writer/retina/SinkContextManager.java | 3 +- .../writer/retina/TableCrossTxWriter.java | 2 + .../sink/writer/retina/TableWriter.java | 37 +++++++++++++------ 4 files changed, 30 insertions(+), 16 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 435e583..e3ed72b 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -2,7 +2,8 @@ sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=retina -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/memory_index.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/update_rocksdb_index.csv +sink.retina.bucket=16 sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -21,7 +22,6 @@ sink.csv.enable_header=false sink.retina.embedded=false # stub or stream sink.retina.mode=stream -sink.retina.bucket=16 #writer.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index c9a1aa7..79a4070 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -202,6 +202,7 @@ private void handleOrphanEvents(SinkContext ctx) throws SinkException protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException { + event.setTimeStamp(ctx.getTimestamp()); event.initIndexKey(); switch (event.getOp()) { @@ -246,7 +247,6 @@ protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws private boolean writeBeforeEvent(SinkContext ctx, RowChangeEvent event) { String table = event.getTable(); - event.setTimeStamp(ctx.getTimestamp()); long tableId = event.getTableId(); int beforeBucketFromIndex = event.getBeforeBucketFromIndex(); return tableWriterProxy.getTableWriter(table, tableId, beforeBucketFromIndex).write(event, ctx); @@ -255,7 +255,6 @@ private boolean writeBeforeEvent(SinkContext ctx, RowChangeEvent event) private boolean writeAfterEvent(SinkContext ctx, RowChangeEvent event) { String table = event.getTable(); - event.setTimeStamp(ctx.getTimestamp()); long tableId = event.getTableId(); int afterBucketFromIndex = event.getAfterBucketFromIndex(); return tableWriterProxy.getTableWriter(table, tableId, afterBucketFromIndex).write(event, ctx); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 2057cfb..74f6808 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -91,6 +91,7 @@ public void flush() { if (smallBatch != null && !smallBatch.isEmpty()) { + // tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucketId(bucketId).setTxId(currentTxId).build()); tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); tableUpdateCount.add(smallBatch.size()); } @@ -104,6 +105,7 @@ public void flush() if (smallBatch != null) { + // tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucketId(bucketId).setTxId(currentTxId).build()); tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); tableUpdateCount.add(smallBatch.size()); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 94b877a..9db3006 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -75,19 +75,32 @@ protected TableWriter(String tableName) protected static void addUpdateData(RowChangeEvent rowChangeEvent, RetinaProto.TableUpdateData.Builder builder) throws SinkException { - if (rowChangeEvent.hasBeforeData()) + switch (rowChangeEvent.getOp()) { - RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); - deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); - builder.addDeleteData(deleteDataBuilder); - } - - if (rowChangeEvent.hasAfterData()) - { - RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); - insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); - insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); - builder.addInsertData(insertDataBuilder); + case SNAPSHOT, INSERT -> + { + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); + insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); + insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); + builder.addInsertData(insertDataBuilder); + } + case UPDATE -> + { + RetinaProto.UpdateData.Builder updateDataBuilder = RetinaProto.UpdateData.newBuilder(); + updateDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); + updateDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); + builder.addUpdateData(updateDataBuilder); + } + case DELETE -> + { + RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); + deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); + builder.addDeleteData(deleteDataBuilder); + } + case UNRECOGNIZED -> + { + throw new SinkException("Unrecognized op: " + rowChangeEvent.getOp()); + } } } From d924bf37803ccaf2da1c677bcfbd44acce74a232 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Mon, 10 Nov 2025 03:51:29 +0000 Subject: [PATCH 15/53] transaction close --- conf/pixels-sink.aws.properties | 10 +++++----- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 16 ++++++++++++++- .../sink/writer/retina/RetinaWriter.java | 3 --- .../sink/writer/retina/TransactionProxy.java | 20 ++++++++++++++++++- 4 files changed, 39 insertions(+), 10 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index e3ed72b..77e8d56 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -2,8 +2,8 @@ sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=retina -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/update_rocksdb_index.csv -sink.retina.bucket=16 +sink.retina.bucket=1 +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_opt1.csv sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -35,7 +35,7 @@ sink.commit.batch.worker=32 sink.commit.batch.delay=200 ## Proto Config sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp -sink.proto.data=data +sink.proto.data=hybench10 sink.proto.maxRecords=1000000 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 @@ -48,7 +48,7 @@ sink.trans.batch.size=200 sink.trans.mode=batch # Sink Metrics sink.monitor.enable=true -sink.monitor.port=9464 +sink.monitor.port=9465 sink.monitor.report.interval=5000 # Interact with other rpc sink.rpc.enable=true @@ -66,7 +66,7 @@ debezium.database.hostname=realtime-pg-2 debezium.database.port=5432 debezium.database.user=pixels debezium.database.password=pixels_realtime_crud -debezium.database.dbname=pixels_bench_sf1x +debezium.database.dbname=pixels_bench_sf10x debezium.plugin.name=pgoutput debezium.database.server.id=1 debezium.schema.include.list=public diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 5473f34..f85a7f4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -22,6 +22,8 @@ import io.pixelsdb.pixels.sink.source.SinkSource; import io.pixelsdb.pixels.sink.source.SinkSourceFactory; import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriterFactory; import io.pixelsdb.pixels.sink.writer.retina.TransactionProxy; import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.hotspot.DefaultExports; @@ -44,7 +46,7 @@ public static void main(String[] args) throws IOException { Runtime.getRuntime().addShutdownHook(new Thread(() -> { - TransactionProxy.Instance().close(); + TransactionProxy.staticClose(); sinkSource.stopProcessor(); LOGGER.info("Pixels Sink Server shutdown complete"); if (prometheusHttpServer != null) @@ -52,6 +54,18 @@ public static void main(String[] args) throws IOException prometheusHttpServer.close(); } MetricsFacade.getInstance().stop(); + PixelsSinkWriter pixelsSinkWriter = PixelsSinkWriterFactory.getWriter(); + if(pixelsSinkWriter != null) + { + try + { + pixelsSinkWriter.close(); + } catch (IOException e) + { + throw new RuntimeException(e); + } + } + })); init(args); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index d1022e1..5c87d98 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -46,11 +46,8 @@ public class RetinaWriter implements PixelsSinkWriter private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(1024); private final ScheduledExecutorService timeoutScheduler = Executors.newSingleThreadScheduledExecutor(); - private final TransactionProxy transactionProxy = TransactionProxy.Instance(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - private final SinkContextManager sinkContextManager; public RetinaWriter() diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index 8b74456..d99b220 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -39,7 +39,7 @@ public class TransactionProxy { private static final Logger LOGGER = LoggerFactory.getLogger(TransactionProxy.class); - private final static TransactionProxy instance = new TransactionProxy(); + private static volatile TransactionProxy instance; private final TransService transService; private final Queue transContextQueue; private final Object batchLock = new Object(); @@ -82,6 +82,16 @@ private TransactionProxy() public static TransactionProxy Instance() { + if(instance == null) + { + synchronized (TransactionProxy.class) + { + if(instance == null) + { + instance = new TransactionProxy(); + } + } + } return instance; } @@ -207,4 +217,12 @@ public void close() } } } + + public static void staticClose() + { + if(instance != null) + { + instance.close(); + } + } } From f1464e0e62e0d23cb4c3e69efbf8663ff1a75e99 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Mon, 10 Nov 2025 05:35:15 +0000 Subject: [PATCH 16/53] add trace log --- conf/pixels-sink.aws.properties | 4 +- .../writer/retina/SinkContextManager.java | 37 ++++++++++++++----- .../writer/retina/TableCrossTxWriter.java | 9 +++-- .../writer/retina/TableSingleTxWriter.java | 4 +- .../sink/writer/retina/TableWriter.java | 10 ++--- src/main/resources/log4j2.properties | 4 +- 6 files changed, 43 insertions(+), 25 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 77e8d56..6f11599 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -3,7 +3,7 @@ sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=retina sink.retina.bucket=1 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_opt1.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_opt2.csv sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -26,7 +26,7 @@ sink.retina.mode=stream sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 -sink.flush.interval.ms=2000 +sink.flush.interval.ms=5000 sink.flush.batch.size=500 sink.max.retries=3 ## writer commit diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 79a4070..3b4479b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -63,18 +63,31 @@ protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean can { if (sinkContext == null) { + LOGGER.trace("Allocate new tx {}\torder:{}", sourceTxId, event.getTransaction().getTotalOrder()); SinkContext newSinkContext = new SinkContext(sourceTxId); newSinkContext.bufferOrphanedEvent(event); return newSinkContext; } else { - if (sinkContext.getPixelsTransCtx() == null) + try { - sinkContext.bufferOrphanedEvent(event); + sinkContext.getLock().lock(); + if (sinkContext.getPixelsTransCtx() == null) + { + LOGGER.trace("Buffer in tx {}\torder:{}", sourceTxId, event.getTransaction().getTotalOrder()); + canWrite.set(false); + sinkContext.bufferOrphanedEvent(event); + return sinkContext; + } + LOGGER.trace("Ready to write in tx {}\torder:{}", sourceTxId, event.getTransaction().getTotalOrder()); + canWrite.set(true); return sinkContext; + } finally + { + sinkContext.getCond().signalAll(); + sinkContext.getLock().unlock(); } - canWrite.set(true); - return sinkContext; + } }); } @@ -87,6 +100,7 @@ protected void startTransSync(String sourceTxId) { if (oldCtx == null) { + LOGGER.trace("Start trans {} without buffered events", sourceTxId); return new SinkContext(sourceTxId, transactionProxy.getNewTransContext()); } else { @@ -193,6 +207,7 @@ private void handleOrphanEvents(SinkContext ctx) throws SinkException if (buffered != null) { + LOGGER.trace("Handle Orphan Events in {}", ctx.sourceTxId); for (RowChangeEvent event : buffered) { writeRowChangeEvent(ctx, event); @@ -246,20 +261,22 @@ protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws private boolean writeBeforeEvent(SinkContext ctx, RowChangeEvent event) { - String table = event.getTable(); - long tableId = event.getTableId(); int beforeBucketFromIndex = event.getBeforeBucketFromIndex(); - return tableWriterProxy.getTableWriter(table, tableId, beforeBucketFromIndex).write(event, ctx); + return writeBucketEvent(ctx, event, beforeBucketFromIndex); } private boolean writeAfterEvent(SinkContext ctx, RowChangeEvent event) { - String table = event.getTable(); - long tableId = event.getTableId(); int afterBucketFromIndex = event.getAfterBucketFromIndex(); - return tableWriterProxy.getTableWriter(table, tableId, afterBucketFromIndex).write(event, ctx); + return writeBucketEvent(ctx, event, afterBucketFromIndex); } + private boolean writeBucketEvent(SinkContext ctx, RowChangeEvent event, int bucketId) + { + String table = event.getTable(); + long tableId = event.getTableId(); + return tableWriterProxy.getTableWriter(table, tableId, bucketId).write(event, ctx); + } protected SinkContext getSinkContext(String txId) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 74f6808..e3b9cd6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -59,7 +59,7 @@ public TableCrossTxWriter(String t, int bucketId) public void flush() { List batch; - lock.lock(); + bufferLock.lock(); try { if (buffer.isEmpty()) @@ -71,14 +71,14 @@ public void flush() buffer = new LinkedList<>(); } finally { - lock.unlock(); + bufferLock.unlock(); } writeLock.lock(); try { + // TODO(AntiO2) Fix: At high flush rates, the future task may encounter concurrency issues. String txId = null; - String schemaName = null; List smallBatch = null; List txIds = new ArrayList<>(); List fullTableName = new ArrayList<>(); @@ -91,7 +91,6 @@ public void flush() { if (smallBatch != null && !smallBatch.isEmpty()) { - // tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucketId(bucketId).setTxId(currentTxId).build()); tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); tableUpdateCount.add(smallBatch.size()); } @@ -141,6 +140,7 @@ private void failCtxs(List txIds) private void updateCtxCounters(List txIds, List fullTableName, List tableUpdateCount) { + writeLock.lock(); for (int i = 0; i < txIds.size(); i++) { metricsFacade.recordRowEvent(tableUpdateCount.get(i)); @@ -148,6 +148,7 @@ private void updateCtxCounters(List txIds, List fullTableName, L SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); } + writeLock.unlock(); } private RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java index 49c3cc6..29fe27d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java @@ -46,7 +46,7 @@ public void flush() String txId; RetinaProto.TableUpdateData.Builder toBuild; SinkContext sinkContext = null; - lock.lock(); + bufferLock.lock(); try { if (buffer.isEmpty() || currentTxId == null) @@ -78,7 +78,7 @@ public void flush() throw new RuntimeException(e); } finally { - lock.unlock(); + bufferLock.unlock(); } RowChangeEvent event1 = batch.get(0); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 9db3006..43f8324 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -47,7 +47,7 @@ public abstract class TableWriter protected final RetinaServiceProxy delegate; // physical writer protected final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); - protected final ReentrantLock lock = new ReentrantLock(); + protected final ReentrantLock bufferLock = new ReentrantLock(); protected final String tableName; protected final long flushInterval; // Shared state (protected by lock) @@ -110,7 +110,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) { try { - lock.lock(); + bufferLock.lock(); try { txId = ctx.getSourceTxId(); @@ -140,7 +140,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) { try { - lock.lock(); + bufferLock.lock(); try { if (txId.equals(currentTxId)) @@ -149,7 +149,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) } } finally { - lock.unlock(); + bufferLock.unlock(); } } catch (Exception e) { @@ -158,7 +158,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) }, flushInterval, TimeUnit.MILLISECONDS); } finally { - lock.unlock(); + bufferLock.unlock(); } return true; } catch (Exception e) diff --git a/src/main/resources/log4j2.properties b/src/main/resources/log4j2.properties index f385702..a7f4959 100644 --- a/src/main/resources/log4j2.properties +++ b/src/main/resources/log4j2.properties @@ -1,7 +1,7 @@ status=info name=pixels-sink filter.threshold.type=ThresholdFilter -filter.threshold.level=info +filter.threshold.level=trace appender.console.type=Console appender.console.name=STDOUT appender.console.layout.type=PatternLayout @@ -12,7 +12,7 @@ appender.rolling.append=true appender.rolling.fileName=${env:PIXELS_HOME}/logs/pixels-sink.log appender.rolling.layout.type=PatternLayout appender.rolling.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n -rootLogger.level=info +rootLogger.level=trace rootLogger.appenderRef.stdout.ref=STDOUT rootLogger.appenderRef.log.ref=log logger.transaction.name=io.pixelsdb.pixels.sink.sink.retina.RetinaWriter From a9795ea29d963b7587192900f24064c0afdf9f52 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 11 Nov 2025 03:55:35 +0000 Subject: [PATCH 17/53] feature: limit active txns --- conf/pixels-sink.aws.properties | 4 +- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 4 +- .../pixels/sink/util/BlockingBoundedMap.java | 128 ++++++++++++++++++ .../pixels/sink/util/MetricsFacade.java | 15 +- .../sink/writer/retina/SinkContext.java | 2 +- .../writer/retina/SinkContextManager.java | 29 +++- src/main/resources/log4j2.properties | 4 +- 7 files changed, 171 insertions(+), 15 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 6f11599..8e9f5ea 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -2,8 +2,8 @@ sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=retina -sink.retina.bucket=1 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_opt2.csv +sink.retina.bucket=4 +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_level_b4.csv sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index f85a7f4..9892a25 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -24,6 +24,7 @@ import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriterFactory; +import io.pixelsdb.pixels.sink.writer.retina.SinkContextManager; import io.pixelsdb.pixels.sink.writer.retina.TransactionProxy; import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.hotspot.DefaultExports; @@ -91,6 +92,7 @@ private static void init(String[] args) throws IOException { CommandLineConfig cmdLineConfig = new CommandLineConfig(args); PixelsSinkConfigFactory.initialize(cmdLineConfig.getConfigPath()); - MetricsFacade.getInstance(); + MetricsFacade metricsFacade = MetricsFacade.getInstance(); + metricsFacade.setSinkContextManager(SinkContextManager.getInstance()); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java new file mode 100644 index 0000000..67bde68 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java @@ -0,0 +1,128 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.util; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Semaphore; +import java.util.function.BiFunction; + +/** + * A thread-safe bounded map that blocks when full. + *

+ * Similar to ConcurrentHashMap, but with a capacity limit. + * When the map reaches its maximum size, any new insertion or compute + * for a new key will block until space becomes available. + */ +public class BlockingBoundedMap +{ + private final int maxSize; + private final Semaphore semaphore; + private final ConcurrentMap map; + + public BlockingBoundedMap(int maxSize) + { + this.maxSize = maxSize; + this.map = new ConcurrentHashMap<>(); + this.semaphore = new Semaphore(maxSize); + } + + /** + * Puts a key-value pair into the map. + * If the map is full, this call blocks until space becomes available. + */ + private void put(K key, V value) throws InterruptedException + { + semaphore.acquire(); // block if full + V prev = map.put(key, value); + if (prev != null) + { + // replaced existing value — no new space consumed + semaphore.release(); + } + } + + public V get(K key) + { + return map.get(key); + } + + /** + * Removes a key from the map and releases one permit if a value was present. + */ + public V remove(K key) + { + V val = map.remove(key); + if (val != null) + { + semaphore.release(); + } + return val; + } + + public int size() + { + return map.size(); + } + + /** + * Atomically computes a new value for a key, blocking if capacity is full. + *

+ * - If the key is new and capacity is full, this method blocks until space is freed. + * - If the key already exists, it does not block. + * - If the remapping function returns null, the key is removed and capacity is released. + */ + public V compute(K key, BiFunction remappingFunction) + { + for (; ; ) + { + V oldVal = map.get(key); + if (oldVal == null) + { + try + { + semaphore.acquire(); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + return null; + } + + V newVal = remappingFunction.apply(key, null); + if (newVal == null) + { + semaphore.release(); + return null; + } + + V existing = map.putIfAbsent(key, newVal); + if (existing == null) + { + return newVal; + } else + { + semaphore.release(); + continue; + } + } else + { + return map.compute(key, remappingFunction); + } + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 463bf75..7b9475c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -21,8 +21,10 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.writer.retina.SinkContextManager; import io.prometheus.client.Counter; import io.prometheus.client.Summary; +import lombok.Setter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,6 +59,10 @@ public class MetricsFacade private final String monitorReportPath; private final AtomicBoolean running = new AtomicBoolean(false); + + @Setter + private SinkContextManager sinkContextManager; + private Thread reportThread; private long lastRowChangeCount = 0; @@ -188,7 +194,6 @@ private MetricsFacade(boolean enabled) monitorReportEnabled = config.isMonitorReportEnabled(); monitorReportInterval = config.getMonitorReportInterval(); monitorReportPath = config.getMonitorReportFile(); - if (monitorReportEnabled) { running.set(true); @@ -206,6 +211,7 @@ private static synchronized void initialize() if (instance == null) { instance = new MetricsFacade(config.isMonitorEnabled()); + LOGGER.info("Init Metrics Facade"); } } @@ -360,8 +366,8 @@ public void run() { try { - logPerformance(); Thread.sleep(monitorReportInterval); + logPerformance(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -405,13 +411,14 @@ public void logPerformance() LOGGER.info( "Performance report: +{} rows (+{}/s), +{} transactions (+{}/s), +{} debezium (+{}/s)" + ", +{} serdRows (+{}/s), +{} serdTxs (+{}/s)" + - " in {} ms", + " in {} ms\t activeTxNum: {}", deltaRows, String.format("%.2f", rowOips), deltaTxns, String.format("%.2f", txnOips), deltaDebezium, String.format("%.2f", dbOips), deltaSerdRows, String.format("%.2f", serdRowsOips), deltaSerdTxs, String.format("%.2f", serdTxsOips), - monitorReportInterval + monitorReportInterval, + sinkContextManager.getActiveTxnsNum() ); String time = LocalDateTime.now().format(DateTimeFormatter.ofPattern("HH:mm:ss")); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 5bb74cf..ee4b593 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -52,7 +52,7 @@ public class SinkContext @Getter final String sourceTxId; @Getter - final Map tableCounters = new ConcurrentHashMap<>(); + Map tableCounters = new ConcurrentHashMap<>(); @Getter final AtomicInteger pendingEvents = new AtomicInteger(0); @Getter diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 3b4479b..42d0236 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -23,6 +23,7 @@ import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.util.BlockingBoundedMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,10 +39,10 @@ public class SinkContextManager private static final Logger LOGGER = LoggerFactory.getLogger(SinkContextManager.class); private static final Logger BUCKET_TRACE_LOGGER = LoggerFactory.getLogger("bucket_trace"); - private final static SinkContextManager INSTANCE = new SinkContextManager(); - private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(); + private final BlockingBoundedMap activeTxContexts = new BlockingBoundedMap<>(100000); + // private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(10000); private final TransactionProxy transactionProxy = TransactionProxy.Instance(); private final TransService transService = TransService.Instance(); private final TableWriterProxy tableWriterProxy; @@ -51,9 +52,17 @@ private SinkContextManager() this.tableWriterProxy = TableWriterProxy.getInstance(); } - public static SinkContextManager getInstance() - { - return INSTANCE; + private static volatile SinkContextManager instance; + + public static SinkContextManager getInstance() { + if (instance == null) { + synchronized (SinkContextManager.class) { + if (instance == null) { + instance = new SinkContextManager(); + } + } + } + return instance; } protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean canWrite) @@ -107,6 +116,11 @@ protected void startTransSync(String sourceTxId) oldCtx.getLock().lock(); try { + if(oldCtx.getPixelsTransCtx() != null) + { + LOGGER.warn("Previous tx {} has been released, maybe due to loop process", sourceTxId); + oldCtx.tableCounters = new ConcurrentHashMap<>(); + } oldCtx.setPixelsTransCtx(transactionProxy.getNewTransContext()); handleOrphanEvents(oldCtx); oldCtx.getCond().signalAll(); @@ -292,4 +306,9 @@ protected void writeRowChangeEvent(String randomId, RowChangeEvent event) throws { writeRowChangeEvent(getSinkContext(randomId), event); } + + public int getActiveTxnsNum() + { + return activeTxContexts.size(); + } } diff --git a/src/main/resources/log4j2.properties b/src/main/resources/log4j2.properties index a7f4959..f385702 100644 --- a/src/main/resources/log4j2.properties +++ b/src/main/resources/log4j2.properties @@ -1,7 +1,7 @@ status=info name=pixels-sink filter.threshold.type=ThresholdFilter -filter.threshold.level=trace +filter.threshold.level=info appender.console.type=Console appender.console.name=STDOUT appender.console.layout.type=PatternLayout @@ -12,7 +12,7 @@ appender.rolling.append=true appender.rolling.fileName=${env:PIXELS_HOME}/logs/pixels-sink.log appender.rolling.layout.type=PatternLayout appender.rolling.layout.pattern=%-d{yyyy-MM-dd HH:mm:ss,SSS} [%c]-[%p] %m%n -rootLogger.level=trace +rootLogger.level=info rootLogger.appenderRef.stdout.ref=STDOUT rootLogger.appenderRef.log.ref=log logger.transaction.name=io.pixelsdb.pixels.sink.sink.retina.RetinaWriter From 1b628220f43fee52faeaa84ea0f9a78b9a4428d9 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 11 Nov 2025 05:33:35 +0000 Subject: [PATCH 18/53] feature: record freshness stats --- conf/pixels-sink.aws.properties | 4 +-- pom.xml | 7 ++++- .../pixels/sink/util/MetricsFacade.java | 30 ++++++++++++++++-- .../sink/writer/retina/SinkContext.java | 4 +++ .../writer/retina/SinkContextManager.java | 4 ++- .../sink/writer/retina/TransactionProxy.java | 31 +++++++++++++------ 6 files changed, 64 insertions(+), 16 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 8e9f5ea..5fbf81f 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -2,8 +2,8 @@ sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=retina -sink.retina.bucket=4 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_level_b4.csv +sink.retina.bucket=2 +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_level_b2.csv sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 diff --git a/pom.xml b/pom.xml index 886e992..e82fbbb 100644 --- a/pom.xml +++ b/pom.xml @@ -210,7 +210,6 @@ 2.6.2.Final - io.prometheus @@ -229,6 +228,12 @@ 0.16.0 + + org.apache.commons + commons-math3 + 3.6.1 + + io.pixelsdb diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 7b9475c..16d47eb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -25,6 +25,7 @@ import io.prometheus.client.Counter; import io.prometheus.client.Summary; import lombok.Setter; +import org.apache.commons.math3.stat.descriptive.SynchronizedDescriptiveStatistics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +57,9 @@ public class MetricsFacade private final Summary totalLatency; private final boolean monitorReportEnabled; private final int monitorReportInterval; + + private final SynchronizedDescriptiveStatistics freshness; + private final String monitorReportPath; private final AtomicBoolean running = new AtomicBoolean(false); @@ -63,7 +67,7 @@ public class MetricsFacade @Setter private SinkContextManager sinkContextManager; - private Thread reportThread; + private final Thread reportThread; private long lastRowChangeCount = 0; private long lastTransactionCount = 0; @@ -173,6 +177,7 @@ private MetricsFacade(boolean enabled) .quantile(0.99, 0.001) .register(); + this.freshness = new SynchronizedDescriptiveStatistics(); } else { this.debeziumEventCounter = null; @@ -189,6 +194,7 @@ private MetricsFacade(boolean enabled) this.retinaServiceLatency = null; this.writerLatency = null; this.totalLatency = null; + this.freshness = null; } monitorReportEnabled = config.isMonitorReportEnabled(); @@ -359,6 +365,13 @@ public int getTransactionEvent() return (int) transactionCounter.get(); } + public void recordFreshness(double freshnessMill) + { + if(enabled && freshness != null) + { + freshness.addValue(freshnessMill); + } + } public void run() { @@ -379,7 +392,6 @@ public void run() } } - public void logPerformance() { long currentRows = (long) rowEventCounter.get(); @@ -421,6 +433,20 @@ public void logPerformance() sinkContextManager.getActiveTxnsNum() ); + LOGGER.info( + String.format( + "Freshness Report: Count=%d, Max=%.2f, Min=%.2f, Mean=%.2f, P50=%.2f, P90=%.2f, P95=%.2f, P99=%.2f", + freshness.getN(), + freshness.getMax(), + freshness.getMin(), + freshness.getMean(), + freshness.getPercentile(50), + freshness.getPercentile(90), + freshness.getPercentile(95), + freshness.getPercentile(99) + ) + ); + String time = LocalDateTime.now().format(DateTimeFormatter.ofPattern("HH:mm:ss")); // Append to CSV for plotting try (FileWriter fw = new FileWriter(monitorReportPath, true)) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index ee4b593..86c4be2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -69,6 +69,10 @@ public class SinkContext @Getter private boolean failed = false; + @Getter + @Setter + private long startTime; + public SinkContext(String sourceTxId) { this.sourceTxId = sourceTxId; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 42d0236..687a664 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -148,6 +148,8 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) return; } + ctx.setStartTime(System.currentTimeMillis()); + try { try @@ -175,7 +177,7 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) if (!failed) { LOGGER.trace("Committed transaction: {}", txId); - transactionProxy.commitTransAsync(ctx.getPixelsTransCtx()); + transactionProxy.commitTransAsync(ctx); } else { LOGGER.info("Abort transaction: {}", txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index d99b220..cbbda6b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -45,7 +45,7 @@ public class TransactionProxy private final Object batchLock = new Object(); private final ExecutorService batchCommitExecutor; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final BlockingQueue toCommitTransContextQueue; + private final BlockingQueue toCommitTransContextQueue; private final int BATCH_SIZE; private final int WORKER_COUNT; @@ -130,7 +130,7 @@ public TransContext getNewTransContext() } } - public void commitTransAsync(TransContext transContext) + public void commitTransAsync(SinkContext transContext) { toCommitTransContextQueue.add(transContext); } @@ -139,18 +139,20 @@ private void batchCommitWorker() { List batchTransIds = new ArrayList<>(BATCH_SIZE); List batchContexts = new ArrayList<>(BATCH_SIZE); - + List txStartTimes = new ArrayList<>(BATCH_SIZE); while (true) { try { batchContexts.clear(); batchTransIds.clear(); + txStartTimes.clear(); - TransContext first = toCommitTransContextQueue.take(); - batchContexts.add(first); - batchTransIds.add(first.getTransId()); - + SinkContext firstSinkContext = toCommitTransContextQueue.take(); + TransContext transContext = firstSinkContext.getPixelsTransCtx(); + batchContexts.add(transContext); + batchTransIds.add(transContext.getTransId()); + txStartTimes.add(firstSinkContext.getStartTime()); long startTime = System.nanoTime(); while (batchContexts.size() < BATCH_SIZE) @@ -162,17 +164,26 @@ private void batchCommitWorker() break; } - TransContext ctx = toCommitTransContextQueue.poll(remainingMs, TimeUnit.MILLISECONDS); + SinkContext ctx = toCommitTransContextQueue.poll(remainingMs, TimeUnit.MILLISECONDS); if (ctx == null) { break; } - batchContexts.add(ctx); - batchTransIds.add(ctx.getTransId()); + transContext = ctx.getPixelsTransCtx(); + batchContexts.add(transContext); + batchTransIds.add(transContext.getTransId()); + txStartTimes.add(ctx.getStartTime()); } transService.commitTransBatch(batchTransIds, false); metricsFacade.recordTransaction(batchTransIds.size()); + long txEndTime = System.currentTimeMillis(); + + txStartTimes.forEach( + txStartTime -> { + metricsFacade.recordFreshness(txEndTime- txStartTime); + } + ); if (LOGGER.isTraceEnabled()) { From 8610c8deff58a4921517e9b3fb3b2a855eba5e3e Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 11 Nov 2025 05:46:18 +0000 Subject: [PATCH 19/53] feature: record rps stats --- .../pixels/sink/util/MetricsFacade.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 16d47eb..d5b3ce9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -59,6 +59,7 @@ public class MetricsFacade private final int monitorReportInterval; private final SynchronizedDescriptiveStatistics freshness; + private final SynchronizedDescriptiveStatistics rowChangeSpeed; private final String monitorReportPath; @@ -178,6 +179,7 @@ private MetricsFacade(boolean enabled) .register(); this.freshness = new SynchronizedDescriptiveStatistics(); + this.rowChangeSpeed = new SynchronizedDescriptiveStatistics(); } else { this.debeziumEventCounter = null; @@ -195,6 +197,7 @@ private MetricsFacade(boolean enabled) this.writerLatency = null; this.totalLatency = null; this.freshness = null; + this.rowChangeSpeed = null; } monitorReportEnabled = config.isMonitorReportEnabled(); @@ -420,6 +423,8 @@ public void logPerformance() double serdRowsOips = deltaSerdRows / seconds; double serdTxsOips = deltaSerdTxs / seconds; + rowChangeSpeed.addValue(rowOips); + LOGGER.info( "Performance report: +{} rows (+{}/s), +{} transactions (+{}/s), +{} debezium (+{}/s)" + ", +{} serdRows (+{}/s), +{} serdTxs (+{}/s)" + @@ -433,6 +438,20 @@ public void logPerformance() sinkContextManager.getActiveTxnsNum() ); + LOGGER.info( + String.format( + "Row Per/Second Summary: Max=%.2f, Min=%.2f, Mean=%.2f, P10=%.2f, P50=%.2f, P90=%.2f, P95=%.2f, P99=%.2f", + rowChangeSpeed.getMax(), + rowChangeSpeed.getMin(), + rowChangeSpeed.getMean(), + rowChangeSpeed.getPercentile(10), + rowChangeSpeed.getPercentile(50), + rowChangeSpeed.getPercentile(90), + rowChangeSpeed.getPercentile(95), + rowChangeSpeed.getPercentile(99) + ) + ); + LOGGER.info( String.format( "Freshness Report: Count=%d, Max=%.2f, Min=%.2f, Mean=%.2f, P50=%.2f, P90=%.2f, P95=%.2f, P99=%.2f", From a67dcac2e02610da54b4495412ff4fd6f823f0ba Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 14 Nov 2025 11:29:29 +0000 Subject: [PATCH 20/53] modify config --- conf/pixels-sink.aws.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 5fbf81f..66164f5 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -3,7 +3,7 @@ sink.datasource=storage # Sink Config: retina | csv | proto | none sink.mode=retina sink.retina.bucket=2 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_level_b2.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_lev_larger_mem.csv sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 From 83d885f17beba66616907d0756cdc94b3774f3a2 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 14 Nov 2025 20:17:23 +0800 Subject: [PATCH 21/53] Enhance Freshness Record --- conf/pixels-sink.aws.properties | 4 + .../pixels/sink/config/PixelsSinkConfig.java | 6 ++ .../pixels/sink/util/MetricsFacade.java | 61 ++++++++++++- .../pixels/sink/util/OneSecondAverage.java | 90 +++++++++++++++++++ 4 files changed, 160 insertions(+), 1 deletion(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 66164f5..a253c15 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -4,6 +4,8 @@ sink.datasource=storage sink.mode=retina sink.retina.bucket=2 sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_lev_larger_mem.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness.csv + sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -50,6 +52,8 @@ sink.trans.mode=batch sink.monitor.enable=true sink.monitor.port=9465 sink.monitor.report.interval=5000 +sink.monitor.freshness.interval=1000 + # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 9cac6ca..b4ab88b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -85,6 +85,12 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.report.interval", defaultValue = "5000") private short monitorReportInterval; + @ConfigKey(value = "sink.monitor.freshness.interval", defaultValue = "1000") + private int freshnessReportInterval; + + @ConfigKey(value = "sink.monitor.freshness.file", defaultValue = "/tmp/sinkFreshness.csv") + private String monitorFreshnessReportFile; + @ConfigKey(value = "sink.monitor.report.file", defaultValue = "/tmp/sink.csv") private String monitorReportFile; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index d5b3ce9..6985728 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -57,11 +57,14 @@ public class MetricsFacade private final Summary totalLatency; private final boolean monitorReportEnabled; private final int monitorReportInterval; + private final int freshnessReportInterval; private final SynchronizedDescriptiveStatistics freshness; private final SynchronizedDescriptiveStatistics rowChangeSpeed; + private final OneSecondAverage freshnessAvg; private final String monitorReportPath; + private final String freshnessReportPath; private final AtomicBoolean running = new AtomicBoolean(false); @@ -69,7 +72,7 @@ public class MetricsFacade private SinkContextManager sinkContextManager; private final Thread reportThread; - + private final Thread freshnessThread; private long lastRowChangeCount = 0; private long lastTransactionCount = 0; private long lastDebeziumCount = 0; @@ -200,6 +203,10 @@ private MetricsFacade(boolean enabled) this.rowChangeSpeed = null; } + freshnessReportInterval = config.getFreshnessReportInterval(); + freshnessReportPath = config.getMonitorFreshnessReportFile(); + freshnessAvg = new OneSecondAverage(freshnessReportInterval); + monitorReportEnabled = config.isMonitorReportEnabled(); monitorReportInterval = config.getMonitorReportInterval(); monitorReportPath = config.getMonitorReportFile(); @@ -209,9 +216,12 @@ private MetricsFacade(boolean enabled) reportThread = new Thread(this::run, "Metrics Report Thread"); LOGGER.info("Metrics Report Thread Started"); reportThread.start(); + freshnessThread = new Thread(this::runFreshness, "Freshness Thread"); + freshnessThread.start(); } else { reportThread = null; + freshnessThread = null; } } @@ -240,6 +250,11 @@ public void stop() { reportThread.interrupt(); } + + if (freshnessThread != null) + { + freshnessThread.interrupt(); + } LOGGER.info("Monitor report thread stopped."); } @@ -374,6 +389,11 @@ public void recordFreshness(double freshnessMill) { freshness.addValue(freshnessMill); } + + if(freshnessAvg != null) + { + freshnessAvg.record(freshnessMill); + } } public void run() @@ -395,6 +415,45 @@ public void run() } } + public void runFreshness() + { + try + { + Thread.sleep(monitorReportInterval); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + while (running.get()) + { + try + { + Thread.sleep(freshnessReportInterval); + try (FileWriter fw = new FileWriter(monitorReportPath, true)) + { + long now = System.currentTimeMillis(); + double avg = freshnessAvg.getWindowAverage(); + if(Double.isNaN(avg)) + { + continue; + } + fw.write(now + "," + avg + "\n"); + fw.flush(); + } catch (IOException e) + { + LOGGER.warn("Failed to write perf metrics: " + e.getMessage()); + } + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + break; + } catch (Throwable t) + { + LOGGER.warn("Error while reporting performance.", t); + } + } + } + public void logPerformance() { long currentRows = (long) rowEventCounter.get(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java b/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java new file mode 100644 index 0000000..c430838 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java @@ -0,0 +1,90 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.pixelsdb.pixels.sink.util; + +import java.util.ArrayDeque; +import java.util.Deque; + +public class OneSecondAverage +{ + /** + * Time window in milliseconds + */ + private final int windowMillis; + + /** + * Sliding window storing timestamped values + */ + private final Deque window = new ArrayDeque<>(); + + /** + * Constructor with configurable window size (milliseconds) + */ + public OneSecondAverage(int windowMillis) + { + this.windowMillis = windowMillis; + } + + /** + * Record a new data point + */ + public synchronized void record(double v) + { + long now = System.currentTimeMillis(); + window.addLast(new TimedValue(now, v)); + evictOld(now); + } + + /** + * Remove all values older than windowMillis + */ + private void evictOld(long now) + { + while (!window.isEmpty() && now - window.peekFirst().timestamp > windowMillis) + { + window.removeFirst(); + } + } + + /** + * Compute average of values in the time window + */ + public synchronized double getWindowAverage() + { + long now = System.currentTimeMillis(); + evictOld(now); + + if (window.isEmpty()) + { + return Double.NaN; + } + + double sum = 0; + for (TimedValue tv : window) + { + sum += tv.value; + } + return sum / window.size(); + } + + /** + * Timestamped data point + */ + private record TimedValue(long timestamp, double value) + { + } +} From bbe60d7be72b9d932ac2e83ec40bde301b6fdd15 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 14 Nov 2025 20:39:52 +0800 Subject: [PATCH 22/53] Add RateLimiter --- conf/pixels-sink.aws.properties | 2 ++ pom.xml | 5 +++++ .../pixels/sink/config/PixelsSinkConfig.java | 15 +++++++++++++-- .../pixels/sink/provider/EventProvider.java | 4 ++-- .../sink/source/AbstractSinkStorageSource.java | 14 ++++++++++++-- 5 files changed, 34 insertions(+), 6 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index a253c15..d1d65a8 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,5 +1,7 @@ # engine | kafka | storage sink.datasource=storage +# -1 means no limit, Only implement in storage source now +sink.datasource.rate.limit=10000 # Sink Config: retina | csv | proto | none sink.mode=retina sink.retina.bucket=2 diff --git a/pom.xml b/pom.xml index e82fbbb..0483d9f 100644 --- a/pom.xml +++ b/pom.xml @@ -141,6 +141,11 @@ opencsv ${dep.opencsv.version} + + com.google.guava + guava + 33.2.0-jre + org.projectlombok diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index b4ab88b..1d4129b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -151,6 +151,11 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.datasource", defaultValue = PixelsSinkDefaultConfig.DATA_SOURCE) private String dataSource; + @ConfigKey(value = "sink.datasource.rate.limit", defaultValue = "-1") + private int sourceRateLimit; + + private boolean enableSourceRateLimit; + @ConfigKey(value = "sink.proto.dir") private String sinkProtoDir; @ConfigKey(value = "sink.proto.data", defaultValue = "data") @@ -167,13 +172,13 @@ public PixelsSinkConfig(String configFilePath) throws IOException { this.config = ConfigFactory.Instance(); this.config.loadProperties(configFilePath); - ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); + init(); } public PixelsSinkConfig(ConfigFactory config) { this.config = config; - ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); + init(); } public String[] getIncludeTables() @@ -181,4 +186,10 @@ public String[] getIncludeTables() return includeTablesRaw.isEmpty() ? new String[0] : includeTablesRaw.split(","); } + private void init() + { + ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); + + this.enableSourceRateLimit = this.sourceRateLimit >= 0; + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index 82a6610..30cfeb9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -35,8 +35,8 @@ public abstract class EventProvider implements private static final long MAX_WAIT_MS = 5; // configurable protected final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(10000); - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(10000); + private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); private Thread providerThread; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 2baa8ef..1398c51 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -1,5 +1,6 @@ package io.pixelsdb.pixels.sink.source; +import com.google.common.util.concurrent.RateLimiter; import io.pixelsdb.pixels.common.physical.PhysicalReader; import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; import io.pixelsdb.pixels.common.physical.Storage; @@ -35,7 +36,6 @@ public abstract class AbstractSinkStorageSource implements SinkSource protected final List files; protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); private final Map consumerThreads = new ConcurrentHashMap<>(); - private final int maxQueueCapacity = 10000; private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); private final Map>> queueMap = new ConcurrentHashMap<>(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); @@ -45,6 +45,8 @@ public abstract class AbstractSinkStorageSource implements SinkSource protected Thread transactionProviderThread; protected Thread transactionProcessorThread; private final boolean storageLoopEnabled; + private final RateLimiter rateLimiter; + private final boolean enableRateLimiter; protected AbstractSinkStorageSource() { @@ -60,6 +62,10 @@ protected AbstractSinkStorageSource() this.transactionProcessor = new TransactionProcessor(transactionEventProvider); this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); + + int sourceRateLimit = pixelsSinkConfig.getSourceRateLimit(); + this.rateLimiter = RateLimiter.create(sourceRateLimit); + this.enableRateLimiter = pixelsSinkConfig.isEnableSourceRateLimit(); } abstract ProtoType getProtoType(int i); @@ -113,7 +119,7 @@ public void start() // Get or create queue BlockingQueue> queue = queueMap.computeIfAbsent(key, - k -> new LinkedBlockingQueue<>(maxQueueCapacity)); + k -> new LinkedBlockingQueue<>()); // Put future in queue queue.put(valueFuture); @@ -176,6 +182,10 @@ private void consumeQueue(int key, BlockingQueue> } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); + if(enableRateLimiter) + { + rateLimiter.acquire(); + } switch (protoType) { case ROW -> handleRowChangeSourceRecord(key, valueBuffer); From 31cbc134e2cf699fba97797543accb814467e117 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 14 Nov 2025 21:00:47 +0800 Subject: [PATCH 23/53] Check Queue --- .../sink/source/AbstractSinkStorageSource.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 1398c51..e24a9d6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -89,8 +89,6 @@ public void start() try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) { long offset = 0; - BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); - BlockingQueue> transQueue = new LinkedBlockingQueue<>(); while (true) { try @@ -137,6 +135,20 @@ public void start() break; } } + + boolean allEmpty = false; + while (!allEmpty) + { + allEmpty = true; + // Check each queue + for (BlockingQueue q : queueMap.values()) + { + if (!q.isEmpty()) { + allEmpty = false; + break; + } + } + } } catch (IOException e) { throw new RuntimeException(e); From 8d0eea73a8ff6e7bcb714b9887400609e2a60aed Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 14 Nov 2025 21:08:46 +0800 Subject: [PATCH 24/53] Fix: freshness report path --- src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 6985728..0066db5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -429,7 +429,7 @@ public void runFreshness() try { Thread.sleep(freshnessReportInterval); - try (FileWriter fw = new FileWriter(monitorReportPath, true)) + try (FileWriter fw = new FileWriter(freshnessReportPath, true)) { long now = System.currentTimeMillis(); double avg = freshnessAvg.getWindowAverage(); From 4ebddb47f1ca0587d0ab8625bd90560eb0dd4c50 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sat, 15 Nov 2025 09:56:48 +0000 Subject: [PATCH 25/53] Perf Freshness --- .gitignore | 3 + conf/jvm.conf | 2 +- conf/pixels-sink.aws.properties | 8 +- perf_freshness.py | 82 ++++++++ requirements.txt | 4 + .../pixels/sink/provider/EventProvider.java | 6 +- .../source/AbstractSinkStorageSource.java | 194 ++++++++---------- 7 files changed, 182 insertions(+), 117 deletions(-) create mode 100644 perf_freshness.py create mode 100644 requirements.txt diff --git a/.gitignore b/.gitignore index 46a9bdd..aed376a 100644 --- a/.gitignore +++ b/.gitignore @@ -51,3 +51,6 @@ resources/*.xml data/ tmp/ + +!requirements.txt +freshness*.png \ No newline at end of file diff --git a/conf/jvm.conf b/conf/jvm.conf index 8dab3af..902190d 100644 --- a/conf/jvm.conf +++ b/conf/jvm.conf @@ -4,7 +4,7 @@ -Duser.timezone=UTC -Xms8g --Xmx16g +-Xmx40g -XX:+UseG1GC -XX:MaxGCPauseMillis=200 diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index d1d65a8..6982aa5 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,12 +1,12 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in storage source now -sink.datasource.rate.limit=10000 +sink.datasource.rate.limit=100000 # Sink Config: retina | csv | proto | none sink.mode=retina sink.retina.bucket=2 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/hybench10_lev_larger_mem.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate50k.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness50k.csv sink.storage.loop=true # Kafka Config @@ -30,7 +30,7 @@ sink.retina.mode=stream sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 -sink.flush.interval.ms=5000 +sink.flush.interval.ms=200 sink.flush.batch.size=500 sink.max.retries=3 ## writer commit diff --git a/perf_freshness.py b/perf_freshness.py new file mode 100644 index 0000000..97267bd --- /dev/null +++ b/perf_freshness.py @@ -0,0 +1,82 @@ +import pandas as pd +import matplotlib.pyplot as plt +import numpy as np + +########################################## +# 配置 CSV 文件 和 标签 +########################################## +csv_files = { + "10k": "tmp/freshness10k.csv", + "20k": "tmp/freshness20k.csv", + "30k": "tmp/freshness30k.csv", + "50k": "tmp/freshness50k.csv", + # "unlimit": "tmp/freshnessut.csv" +} + +MAX_SECONDS = 1800 # 只取前 1800 秒 +BIN_SECONDS = 40 # 可调平均窗口(秒) + +########################################## +# 加载并处理数据 +########################################## +data = {} +for label, path in csv_files.items(): + df = pd.read_csv(path, header=None, names=["ts", "freshness"]) + + # 转为 datetime + df["ts"] = pd.to_datetime(df["ts"], unit="ms") + + # 相对秒 + t0 = df["ts"].iloc[0] + df["sec"] = (df["ts"] - t0).dt.total_seconds() + + # 只取前 MAX_SECONDS 秒 + df = df[df["sec"] <= MAX_SECONDS] + + # 可调平均窗口采样 + df_bin = df.resample(f"{BIN_SECONDS}s", on="ts").mean().reset_index() + + # 时间对齐(横轴) + df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() + + data[label] = df_bin + + +########################################## +# 图 1:按可调窗口采样的时间序列波动 +########################################## +plt.figure(figsize=(10, 5)) + +for label, df in data.items(): + plt.plot(df["bin_sec"], df["freshness"], label=label) + +plt.xlabel("Time (sec)") +plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)") +plt.yscale("log") +plt.title(f"Freshness Over Time ({BIN_SECONDS}-Second Avg, First {MAX_SECONDS}s)") +plt.legend() +plt.tight_layout() +plt.savefig("freshness_over_time_variable_bin.png") +plt.close() + + +########################################## +# 图 2:CDF(使用同样的平均窗口数据) +########################################## +plt.figure(figsize=(10, 5)) + +for label, df in data.items(): + vals = np.sort(df["freshness"].dropna()) + y = np.linspace(0, 1, len(vals)) + plt.plot(vals, y, label=label) + +plt.xlabel(f"Freshness (ms, {BIN_SECONDS}s average)") +plt.xscale("log") +plt.ylabel("CDF") +plt.title(f"Freshness CDF Distribution ({BIN_SECONDS}-Second Sampled)") +plt.legend() +plt.tight_layout() +plt.savefig("freshness_cdf_variable_bin.png") +plt.close() + +print("图已生成: freshness_over_time_variable_bin.png, freshness_cdf_variable_bin.png") diff --git a/requirements.txt b/requirements.txt new file mode 100644 index 0000000..8cb1b70 --- /dev/null +++ b/requirements.txt @@ -0,0 +1,4 @@ +flask +pandas +matplotlib +numpy diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index 30cfeb9..97315db 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -33,10 +33,10 @@ public abstract class EventProvider implements private static final int BATCH_SIZE = 64; private static final int THREAD_NUM = 4; private static final long MAX_WAIT_MS = 5; // configurable - + private static final int MAX_QUEUE_SIZE = 10_000; protected final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(); - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(MAX_QUEUE_SIZE); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(MAX_QUEUE_SIZE); private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); private Thread providerThread; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index e24a9d6..f167867 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -4,7 +4,6 @@ import io.pixelsdb.pixels.common.physical.PhysicalReader; import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; import io.pixelsdb.pixels.common.physical.Storage; -import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; @@ -20,13 +19,13 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; -public abstract class AbstractSinkStorageSource implements SinkSource -{ +public abstract class AbstractSinkStorageSource implements SinkSource { private static final Logger LOGGER = LoggerFactory.getLogger(AbstractSinkStorageSource.class); protected final AtomicBoolean running = new AtomicBoolean(false); @@ -40,16 +39,16 @@ public abstract class AbstractSinkStorageSource implements SinkSource private final Map>> queueMap = new ConcurrentHashMap<>(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final TableProviderAndProcessorPipelineManager tablePipelineManager = new TableProviderAndProcessorPipelineManager(); + private final boolean storageLoopEnabled; + private final RateLimiter rateLimiter; + private final boolean enableRateLimiter; + private final int MAX_QUEUE_SIZE = 10_000; protected TransactionEventStorageProvider transactionEventProvider; protected TransactionProcessor transactionProcessor; protected Thread transactionProviderThread; protected Thread transactionProcessorThread; - private final boolean storageLoopEnabled; - private final RateLimiter rateLimiter; - private final boolean enableRateLimiter; - protected AbstractSinkStorageSource() - { + protected AbstractSinkStorageSource() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); this.topic = pixelsSinkConfig.getSinkProtoData(); this.baseDir = pixelsSinkConfig.getSinkProtoDir(); @@ -70,88 +69,70 @@ protected AbstractSinkStorageSource() abstract ProtoType getProtoType(int i); - protected void handleTransactionSourceRecord(ByteBuffer record) - { + protected void handleTransactionSourceRecord(ByteBuffer record) { transactionEventProvider.putTransRawEvent(record); } @Override - public void start() - { + public void start() { this.running.set(true); this.transactionProcessorThread.start(); this.transactionProviderThread.start(); + List readers = new ArrayList<>(); + for (String file : files) { + Storage.Scheme scheme = Storage.Scheme.fromPath(file); + LOGGER.info("Start read from file {}", file); + PhysicalReader reader = null; + try { + reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); + } catch (IOException e) { + throw new RuntimeException(e); + } + readers.add(reader); + } do { - for (String file : files) - { - Storage.Scheme scheme = Storage.Scheme.fromPath(file); - LOGGER.info("Start read from file {}", file); - try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) - { - long offset = 0; - while (true) - { - try - { - int key, valueLen; - reader.seek(offset); - try - { - key = reader.readInt(ByteOrder.BIG_ENDIAN); - valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) - { - // EOF - break; - } - - ProtoType protoType = getProtoType(key); - offset += Integer.BYTES * 2; - CompletableFuture valueFuture = reader.readAsync(offset, valueLen) - .thenApply(this::copyToHeap) - .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); - // move offset for next record - offset += valueLen; - - - // Get or create queue - BlockingQueue> queue = - queueMap.computeIfAbsent(key, - k -> new LinkedBlockingQueue<>()); - - // Put future in queue - queue.put(valueFuture); - - // Start consumer thread if not exists - consumerThreads.computeIfAbsent(key, k -> - { - Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); - t.setName("consumer-" + key); - t.start(); - return t; - }); - } catch (IOException | InterruptedException e) - { + for (PhysicalReader reader : readers) { + long offset = 0; + while (true) { + try { + int key, valueLen; + reader.seek(offset); + try { + key = reader.readInt(ByteOrder.BIG_ENDIAN); + valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + } catch (IOException e) { + // EOF break; } - } - boolean allEmpty = false; - while (!allEmpty) - { - allEmpty = true; - // Check each queue - for (BlockingQueue q : queueMap.values()) + ProtoType protoType = getProtoType(key); + offset += Integer.BYTES * 2; + CompletableFuture valueFuture = reader.readAsync(offset, valueLen) + .thenApply(this::copyToHeap) + .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); + // move offset for next record + offset += valueLen; + + + // Get or create queue + BlockingQueue> queue = + queueMap.computeIfAbsent(key, + k -> new LinkedBlockingQueue<>(MAX_QUEUE_SIZE)); + + // Put future in queue + queue.put(valueFuture); + + // Start consumer thread if not exists + consumerThreads.computeIfAbsent(key, k -> { - if (!q.isEmpty()) { - allEmpty = false; - break; - } - } + Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); + t.setName("consumer-" + key); + t.start(); + return t; + }); + } catch (IOException | InterruptedException e) { + break; } - } catch (IOException e) - { - throw new RuntimeException(e); } } } while (storageLoopEnabled && isRunning()); @@ -159,11 +140,9 @@ public void start() // signal all queues to stop queueMap.values().forEach(q -> { - try - { + try { q.put(POISON_PILL); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); @@ -171,50 +150,50 @@ public void start() // wait all consumers to finish consumerThreads.values().forEach(t -> { - try - { + try { t.join(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); - } - private void consumeQueue(int key, BlockingQueue> queue, ProtoType protoType) - { - try + // close all readers + for(PhysicalReader reader: readers) { - while (true) - { + try { + reader.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + } + + private void consumeQueue(int key, BlockingQueue> queue, ProtoType protoType) { + try { + while (true) { CompletableFuture value = queue.take(); - if (value == POISON_PILL) - { + if (value == POISON_PILL) { break; } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - if(enableRateLimiter) - { + if (enableRateLimiter) { rateLimiter.acquire(); } - switch (protoType) - { + switch (protoType) { case ROW -> handleRowChangeSourceRecord(key, valueBuffer); case TRANS -> handleTransactionSourceRecord(valueBuffer); } } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); - } catch (ExecutionException e) - { + } catch (ExecutionException e) { LOGGER.error("Error in async processing", e); } } - private ByteBuffer copyToHeap(ByteBuffer directBuffer) - { + private ByteBuffer copyToHeap(ByteBuffer directBuffer) { ByteBuffer duplicate = directBuffer.duplicate(); ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); heapBuffer.put(duplicate); @@ -222,20 +201,17 @@ private ByteBuffer copyToHeap(ByteBuffer directBuffer) return heapBuffer; } - private void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer) - { + private void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer) { tablePipelineManager.routeRecord(key, dataBuffer); } @Override - public boolean isRunning() - { + public boolean isRunning() { return running.get(); } @Override - public void stopProcessor() - { + public void stopProcessor() { running.set(false); transactionProviderThread.interrupt(); transactionProcessorThread.interrupt(); From 0b4bf488e242f8c3a61c3157e48ed0bccb3ce8d4 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 19 Nov 2025 09:50:51 +0000 Subject: [PATCH 26/53] feature: record freshness stats --- conf/pixels-sink.aws.properties | 19 +++--- perf_freshness.py | 38 ++++++++---- .../source/AbstractSinkStorageSource.java | 9 --- .../pixels/sink/util/FlushRateLimiter.java | 60 +++++++++++++++++++ .../sink/writer/retina/SinkContext.java | 18 +++++- .../writer/retina/SinkContextManager.java | 2 +- .../writer/retina/TableCrossTxWriter.java | 11 +++- .../sink/writer/retina/TableWriter.java | 7 ++- .../sink/writer/retina/TransactionProxy.java | 10 ++-- 9 files changed, 132 insertions(+), 42 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 6982aa5..1b9d0a5 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,13 +1,14 @@ # engine | kafka | storage -sink.datasource=storage -# -1 means no limit, Only implement in storage source now -sink.datasource.rate.limit=100000 +sink.datasource=engine +# -1 means no limit, Only implement in retina sink mode yet +sink.datasource.rate.limit=40000 # Sink Config: retina | csv | proto | none -sink.mode=retina +sink.mode=proto sink.retina.bucket=2 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate50k.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness50k.csv - +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/test.csv +# row or txn +sink.monitor.freshness.level=row sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -39,7 +40,7 @@ sink.commit.batch.worker=32 sink.commit.batch.delay=200 ## Proto Config sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp -sink.proto.data=hybench10 +sink.proto.data=hybench_test3 sink.proto.maxRecords=1000000 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 @@ -83,4 +84,4 @@ debezium.topic.prefix=postgresql.oltp_server debezium.transforms=topicRouting debezium.transforms.topicRouting.type=org.apache.kafka.connect.transforms.RegexRouter debezium.transforms.topicRouting.regex=postgresql\\.oltp_server\\.public\\.(.*) -debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf1x.$1 +debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf10x.$1 diff --git a/perf_freshness.py b/perf_freshness.py index 97267bd..d8b48cb 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -6,15 +6,17 @@ # 配置 CSV 文件 和 标签 ########################################## csv_files = { - "10k": "tmp/freshness10k.csv", - "20k": "tmp/freshness20k.csv", - "30k": "tmp/freshness30k.csv", - "50k": "tmp/freshness50k.csv", - # "unlimit": "tmp/freshnessut.csv" + "10k": "tmp/freshness10k_2.csv", + "20k": "tmp/freshness20k_2.csv", + "30k": "tmp/freshness30k_2.csv", + "40k": "tmp/freshness40k_2.csv", + "50k": "tmp/freshness50k_2.csv", + # "50k_old": "tmp/freshness50k.csv", } -MAX_SECONDS = 1800 # 只取前 1800 秒 -BIN_SECONDS = 40 # 可调平均窗口(秒) +MAX_SECONDS = 36000 # 截取前多少秒的数据 +SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) +BIN_SECONDS = 60 # 平均窗口(秒) ########################################## # 加载并处理数据 @@ -30,13 +32,20 @@ t0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t0).dt.total_seconds() + # 跳过前 SKIP_SECONDS 秒 + df = df[df["sec"] >= SKIP_SECONDS] + + # 重新计算时间(所有曲线从 0 秒开始对齐) + t_new0 = df["ts"].iloc[0] + df["sec"] = (df["ts"] - t_new0).dt.total_seconds() + # 只取前 MAX_SECONDS 秒 df = df[df["sec"] <= MAX_SECONDS] # 可调平均窗口采样 df_bin = df.resample(f"{BIN_SECONDS}s", on="ts").mean().reset_index() - # 时间对齐(横轴) + # 对齐横轴(时间序列) df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() data[label] = df_bin @@ -53,7 +62,10 @@ plt.xlabel("Time (sec)") plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)") plt.yscale("log") -plt.title(f"Freshness Over Time ({BIN_SECONDS}-Second Avg, First {MAX_SECONDS}s)") +plt.title( + f"Freshness Over Time ({BIN_SECONDS}-Second Avg, " + f"Skip {SKIP_SECONDS}s, First {MAX_SECONDS}s)" +) plt.legend() plt.tight_layout() plt.savefig("freshness_over_time_variable_bin.png") @@ -61,7 +73,7 @@ ########################################## -# 图 2:CDF(使用同样的平均窗口数据) +# 图 2:CDF(同样使用平均窗口后的数据) ########################################## plt.figure(figsize=(10, 5)) @@ -70,10 +82,12 @@ y = np.linspace(0, 1, len(vals)) plt.plot(vals, y, label=label) +plt.xscale("log") plt.xlabel(f"Freshness (ms, {BIN_SECONDS}s average)") -plt.xscale("log") plt.ylabel("CDF") -plt.title(f"Freshness CDF Distribution ({BIN_SECONDS}-Second Sampled)") +plt.title( + f"Freshness CDF Distribution ({BIN_SECONDS}-Second Sampled, Skip {SKIP_SECONDS}s)" +) plt.legend() plt.tight_layout() plt.savefig("freshness_cdf_variable_bin.png") diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index f167867..7f5a90d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -40,8 +40,6 @@ public abstract class AbstractSinkStorageSource implements SinkSource { private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final TableProviderAndProcessorPipelineManager tablePipelineManager = new TableProviderAndProcessorPipelineManager(); private final boolean storageLoopEnabled; - private final RateLimiter rateLimiter; - private final boolean enableRateLimiter; private final int MAX_QUEUE_SIZE = 10_000; protected TransactionEventStorageProvider transactionEventProvider; protected TransactionProcessor transactionProcessor; @@ -61,10 +59,6 @@ protected AbstractSinkStorageSource() { this.transactionProcessor = new TransactionProcessor(transactionEventProvider); this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); - - int sourceRateLimit = pixelsSinkConfig.getSourceRateLimit(); - this.rateLimiter = RateLimiter.create(sourceRateLimit); - this.enableRateLimiter = pixelsSinkConfig.isEnableSourceRateLimit(); } abstract ProtoType getProtoType(int i); @@ -178,9 +172,6 @@ private void consumeQueue(int key, BlockingQueue> } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - if (enableRateLimiter) { - rateLimiter.acquire(); - } switch (protoType) { case ROW -> handleRowChangeSourceRecord(key, valueBuffer); case TRANS -> handleTransactionSourceRecord(valueBuffer); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java new file mode 100644 index 0000000..f59640f --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java @@ -0,0 +1,60 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.util; + +import com.google.common.util.concurrent.RateLimiter; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; + +public class FlushRateLimiter { + private final RateLimiter rateLimiter; + private final boolean enableRateLimiter; + + private static volatile FlushRateLimiter instance; + + private FlushRateLimiter() + { + PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); + int sourceRateLimit = pixelsSinkConfig.getSourceRateLimit(); + this.rateLimiter = RateLimiter.create(sourceRateLimit); + this.enableRateLimiter = pixelsSinkConfig.isEnableSourceRateLimit(); + } + + public static FlushRateLimiter getInstance() + { + if(instance == null) + { + synchronized (FlushRateLimiter.class) + { + if(instance == null) + { + instance = new FlushRateLimiter(); + } + } + } + return instance; + } + + public void acquire(int num) + { + if (enableRateLimiter) { + rateLimiter.acquire(num); + } + } + +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 86c4be2..42c728b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -70,8 +70,7 @@ public class SinkContext private boolean failed = false; @Getter - @Setter - private long startTime; + private volatile Long startTime = null; public SinkContext(String sourceTxId) { @@ -130,4 +129,19 @@ public void bufferOrphanedEvent(RowChangeEvent event) orphanEvent.add(event); } + public void setCurrStartTime() + { + if(startTime != null) + { + return; + } + + synchronized (this) + { + if(startTime == null) + { + startTime = System.currentTimeMillis(); + } + } + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 687a664..116ce39 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -148,7 +148,7 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) return; } - ctx.setStartTime(System.currentTimeMillis()); + // ctx.setStartTime(System.currentTimeMillis()); try { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index e3b9cd6..402a47e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -45,7 +45,6 @@ public class TableCrossTxWriter extends TableWriter private final int flushBatchSize; private final ReentrantLock writeLock = new ReentrantLock(); private final int bucketId; - public TableCrossTxWriter(String t, int bucketId) { super(t); @@ -77,7 +76,6 @@ public void flush() writeLock.lock(); try { - // TODO(AntiO2) Fix: At high flush rates, the future task may encounter concurrency issues. String txId = null; List smallBatch = null; List txIds = new ArrayList<>(); @@ -109,6 +107,12 @@ public void flush() tableUpdateCount.add(smallBatch.size()); } + flushRateLimiter.acquire(batch.size()); + long txStartTime = System.currentTimeMillis(); + for(String writeTxId: txIds) + { + sinkContextManager.getSinkContext(writeTxId).setCurrStartTime(); + } CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); updateRecordResponseCompletableFuture.thenAccept( @@ -119,6 +123,9 @@ public void flush() failCtxs(txIds); } else { + long txEndTime = System.currentTimeMillis(); + + metricsFacade.recordFreshness(txEndTime- txStartTime); updateCtxCounters(txIds, fullTableName, tableUpdateCount); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 43f8324..ee9c6dc 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -24,6 +24,7 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; @@ -58,7 +59,8 @@ public abstract class TableWriter protected String fullTableName; protected PixelsSinkConfig config; protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); - + protected final FlushRateLimiter flushRateLimiter; + protected final SinkContextManager sinkContextManager; protected TableWriter(String tableName) { @@ -66,7 +68,8 @@ protected TableWriter(String tableName) this.tableName = tableName; this.delegate = new RetinaServiceProxy(); this.flushInterval = config.getFlushIntervalMs(); - + this.flushRateLimiter = FlushRateLimiter.getInstance(); + this.sinkContextManager = SinkContextManager.getInstance(); } /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index cbbda6b..31e9cd8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -179,11 +179,11 @@ private void batchCommitWorker() metricsFacade.recordTransaction(batchTransIds.size()); long txEndTime = System.currentTimeMillis(); - txStartTimes.forEach( - txStartTime -> { - metricsFacade.recordFreshness(txEndTime- txStartTime); - } - ); +// txStartTimes.forEach( +// txStartTime -> { +// metricsFacade.recordFreshness(txEndTime- txStartTime); +// } +// ); if (LOGGER.isTraceEnabled()) { From 30b72243e99db80456593b5a78b2bc43e40331d6 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sun, 23 Nov 2025 08:57:56 +0000 Subject: [PATCH 27/53] Freshness --- conf/pixels-sink.aws.properties | 24 ++++++------ perf_freshness.py | 14 +++---- .../pixels/sink/config/PixelsSinkConfig.java | 5 +++ .../source/AbstractSinkStorageSource.java | 1 + .../sink/writer/retina/RetinaWriter.java | 2 +- .../writer/retina/SinkContextManager.java | 30 ++++++++++++++- .../writer/retina/TableCrossTxWriter.java | 6 ++- .../sink/writer/retina/TableWriter.java | 2 + .../sink/writer/retina/TransactionProxy.java | 37 +++++++++++++++---- 9 files changed, 91 insertions(+), 30 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 1b9d0a5..0eac811 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,12 +1,12 @@ # engine | kafka | storage -sink.datasource=engine +sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=40000 +sink.datasource.rate.limit=20000 # Sink Config: retina | csv | proto | none -sink.mode=proto -sink.retina.bucket=2 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/test.csv +sink.mode=retina +sink.retina.bucket=3 +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate20k_5.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness20k_5.csv # row or txn sink.monitor.freshness.level=row sink.storage.loop=true @@ -31,16 +31,18 @@ sink.retina.mode=stream sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 -sink.flush.interval.ms=200 -sink.flush.batch.size=500 +sink.flush.interval.ms=20000 +sink.flush.batch.size=400 sink.max.retries=3 ## writer commit -sink.commit.batch.size=500 +# sync or async +sink.commit.method=async +sink.commit.batch.size=200 sink.commit.batch.worker=32 -sink.commit.batch.delay=200 +sink.commit.batch.delay=1000 ## Proto Config sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp -sink.proto.data=hybench_test3 +sink.proto.data=hybench10_2 sink.proto.maxRecords=1000000 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 diff --git a/perf_freshness.py b/perf_freshness.py index d8b48cb..8bf183d 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -6,15 +6,15 @@ # 配置 CSV 文件 和 标签 ########################################## csv_files = { - "10k": "tmp/freshness10k_2.csv", - "20k": "tmp/freshness20k_2.csv", - "30k": "tmp/freshness30k_2.csv", - "40k": "tmp/freshness40k_2.csv", - "50k": "tmp/freshness50k_2.csv", - # "50k_old": "tmp/freshness50k.csv", + "10k": "tmp/freshness10k_5.csv", + "20k": "tmp/freshness20k_5.csv", + # "30k": "tmp/freshness30k_2.csv", + "40k": "tmp/freshness40k_5.csv", + "50k": "tmp/freshness50k_5.csv", + "60k": "tmp/freshness60k_5.csv" } -MAX_SECONDS = 36000 # 截取前多少秒的数据 +MAX_SECONDS = 1800 # 截取前多少秒的数据 SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) BIN_SECONDS = 60 # 平均窗口(秒) diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 1d4129b..d7942a4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -46,6 +46,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) private TransactionMode transactionMode; + @ConfigKey(value = "sink.commit.method", defaultValue = "async") + private String commitMethod; + @ConfigKey(value = "sink.commit.batch.size", defaultValue = "500") private int commitBatchSize; @@ -167,6 +170,8 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.storage.loop", defaultValue = "false") private boolean sinkStorageLoop; + @ConfigKey(value = "sink.monitor.freshness.level", defaultValue = "row") // row or txn + private String sinkMonitorFreshnessLevel; public PixelsSinkConfig(String configFilePath) throws IOException { diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 7f5a90d..470eb99 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -86,6 +86,7 @@ public void start() { } do { for (PhysicalReader reader : readers) { + LOGGER.info("Start Read {}", reader.getPath()); long offset = 0; while (true) { try { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 5c87d98..d6c89d9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -43,7 +43,7 @@ public class RetinaWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriter.class); final ExecutorService dispatchExecutor = Executors.newCachedThreadPool(); - private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(1024); + private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(2048); private final ScheduledExecutorService timeoutScheduler = Executors.newSingleThreadScheduledExecutor(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 116ce39..3392881 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -21,6 +21,8 @@ import io.pixelsdb.pixels.common.transaction.TransService; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.util.BlockingBoundedMap; @@ -46,10 +48,25 @@ public class SinkContextManager private final TransactionProxy transactionProxy = TransactionProxy.Instance(); private final TransService transService = TransService.Instance(); private final TableWriterProxy tableWriterProxy; + private final CommitMethod commitMethod; + + private enum CommitMethod + { + Sync, + Async + } private SinkContextManager() { + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.tableWriterProxy = TableWriterProxy.getInstance(); + if(config.getCommitMethod().equals("sync")) + { + this.commitMethod = CommitMethod.Sync; + } else + { + this.commitMethod = CommitMethod.Async; + } } private static volatile SinkContextManager instance; @@ -177,7 +194,18 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) if (!failed) { LOGGER.trace("Committed transaction: {}", txId); - transactionProxy.commitTransAsync(ctx); + switch(commitMethod) + { + case Sync -> + { + transactionProxy.commitTransSync(ctx); + } + case Async -> + { + transactionProxy.commitTransAsync(ctx); + } + } + } else { LOGGER.info("Abort transaction: {}", txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 402a47e..095a033 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -124,8 +124,10 @@ public void flush() } else { long txEndTime = System.currentTimeMillis(); - - metricsFacade.recordFreshness(txEndTime- txStartTime); + if(freshnessLevel.equals("row")) + { + metricsFacade.recordFreshness(txEndTime- txStartTime); + } updateCtxCounters(txIds, fullTableName, tableUpdateCount); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index ee9c6dc..a2ea411 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -61,6 +61,7 @@ public abstract class TableWriter protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); protected final FlushRateLimiter flushRateLimiter; protected final SinkContextManager sinkContextManager; + protected final String freshnessLevel; protected TableWriter(String tableName) { @@ -70,6 +71,7 @@ protected TableWriter(String tableName) this.flushInterval = config.getFlushIntervalMs(); this.flushRateLimiter = FlushRateLimiter.getInstance(); this.sinkContextManager = SinkContextManager.getInstance(); + this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); } /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index 31e9cd8..30c729a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -46,7 +46,7 @@ public class TransactionProxy private final ExecutorService batchCommitExecutor; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final BlockingQueue toCommitTransContextQueue; - + private final String freshnessLevel; private final int BATCH_SIZE; private final int WORKER_COUNT; private final int MAX_WAIT_MS; @@ -77,7 +77,7 @@ private TransactionProxy() batchCommitExecutor.submit(this::batchCommitWorker); } - + this.freshnessLevel = pixelsSinkConfig.getSinkMonitorFreshnessLevel(); } public static TransactionProxy Instance() @@ -135,6 +135,25 @@ public void commitTransAsync(SinkContext transContext) toCommitTransContextQueue.add(transContext); } + public void commitTransSync(SinkContext transContext) + { + try + { + transService.commitTrans(transContext.getPixelsTransCtx().getTransId(), false); + metricsFacade.recordTransaction(); + long txEndTime = System.currentTimeMillis(); + + if(freshnessLevel.equals("txn")) + { + metricsFacade.recordFreshness(txEndTime- transContext.getStartTime()); + } + } + catch (TransException e) + { + LOGGER.error("Batch commit failed: {}", e.getMessage(), e); + } + } + private void batchCommitWorker() { List batchTransIds = new ArrayList<>(BATCH_SIZE); @@ -179,12 +198,14 @@ private void batchCommitWorker() metricsFacade.recordTransaction(batchTransIds.size()); long txEndTime = System.currentTimeMillis(); -// txStartTimes.forEach( -// txStartTime -> { -// metricsFacade.recordFreshness(txEndTime- txStartTime); -// } -// ); - + if(freshnessLevel.equals("txn")) + { + txStartTimes.forEach( + txStartTime -> { + metricsFacade.recordFreshness(txEndTime- txStartTime); + } + ); + } if (LOGGER.isTraceEnabled()) { LOGGER.trace("[{}] Batch committed {} transactions ({} waited ms)", From 8d8047d205ca2d10bed7e8b748801f727e40fe60 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 26 Nov 2025 08:46:14 +0000 Subject: [PATCH 28/53] support multiple nodes --- .gitignore | 3 +- conf/jvm.conf | 2 +- conf/pixels-sink.aws.properties | 12 +- perf_rate.py | 110 ++++++++++++++++++ .../pixels/sink/config/PixelsSinkConfig.java | 4 +- .../pixels/sink/event/RowChangeEvent.java | 10 +- .../writer/retina/RetinaServiceProxy.java | 33 +++++- .../writer/retina/TableCrossTxWriter.java | 2 +- .../writer/retina/TableSingleTxWriter.java | 4 +- .../sink/writer/retina/TableWriter.java | 4 +- .../sink/writer/retina/TableWriterProxy.java | 16 ++- .../pixels/sink/writer/TestRetinaWriter.java | 4 +- 12 files changed, 172 insertions(+), 32 deletions(-) create mode 100644 perf_rate.py diff --git a/.gitignore b/.gitignore index aed376a..15a9737 100644 --- a/.gitignore +++ b/.gitignore @@ -53,4 +53,5 @@ data/ tmp/ !requirements.txt -freshness*.png \ No newline at end of file +freshness*.png +rate*.png diff --git a/conf/jvm.conf b/conf/jvm.conf index 902190d..06600bc 100644 --- a/conf/jvm.conf +++ b/conf/jvm.conf @@ -4,7 +4,7 @@ -Duser.timezone=UTC -Xms8g --Xmx40g +-Xmx45g -XX:+UseG1GC -XX:MaxGCPauseMillis=200 diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 0eac811..a2c079c 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,12 +1,12 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=20000 +sink.datasource.rate.limit=150000 # Sink Config: retina | csv | proto | none sink.mode=retina -sink.retina.bucket=3 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate20k_5.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness20k_5.csv +sink.retina.client=2 +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate150k_6.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness150k_6.csv # row or txn sink.monitor.freshness.level=row sink.storage.loop=true @@ -32,7 +32,7 @@ sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 sink.flush.interval.ms=20000 -sink.flush.batch.size=400 +sink.flush.batch.size=500 sink.max.retries=3 ## writer commit # sync or async @@ -56,7 +56,7 @@ sink.trans.mode=batch # Sink Metrics sink.monitor.enable=true sink.monitor.port=9465 -sink.monitor.report.interval=5000 +sink.monitor.report.interval=1000 sink.monitor.freshness.interval=1000 # Interact with other rpc diff --git a/perf_rate.py b/perf_rate.py new file mode 100644 index 0000000..09a92d7 --- /dev/null +++ b/perf_rate.py @@ -0,0 +1,110 @@ +import pandas as pd +import matplotlib.pyplot as plt +import numpy as np +from datetime import datetime, date + +########################################## +# 配置 CSV 文件 和 标签 +########################################## +csv_files = { + "150k_2node": "tmp/rate150k_6.csv", +} + +# CSV 文件列名 (无表头) +COL_NAMES = ["time", "rows", "txns", "debezium", "serdRows", "serdTxs"] +PLOT_COL = "rows" + +MAX_SECONDS = 1800 # 截取前多少秒的数据 +SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) +BIN_SECONDS = 60 # 平均窗口(秒) + +########################################## +# 加载并处理数据 +########################################## +data = {} +for label, path in csv_files.items(): + # 假设文件是逗号分隔 + df = pd.read_csv( + path, + header=None, + names=COL_NAMES, + sep=',' # 假设是逗号分隔,如果不是请修改 + ) + + # --- 时间戳处理 --- + # 1. 解析 HH:MM:SS 时间字符串 + # 2. 由于没有日期,将其与当前日期合并,形成完整的 datetime 对象 (ts) + df["ts"] = pd.to_datetime(df["time"], format="%H:%M:%S", errors='coerce') + df["ts"] = df["ts"].dt.time.apply(lambda x: datetime.combine(date.today(), x)) + + # 相对秒 (使用第一个时间戳作为 t0) + t0 = df["ts"].iloc[0] + df["sec"] = (df["ts"] - t0).dt.total_seconds() + + # 跳过前 SKIP_SECONDS 秒 + df = df[df["sec"] >= SKIP_SECONDS].copy() # 使用 .copy() 避免 SettingWithCopyWarning + + # 重新计算时间(所有曲线从 0 秒开始对齐) + t_new0 = df["ts"].iloc[0] + df["sec"] = (df["ts"] - t_new0).dt.total_seconds() + + # 只取前 MAX_SECONDS 秒 + df = df[df["sec"] <= MAX_SECONDS] + + # 可调平均窗口采样 (将时间序列设为索引进行重采样) + df = df.set_index("ts") + df_bin = df.resample(f"{BIN_SECONDS}s").mean().reset_index() + + # 对齐横轴(时间序列) + df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() + + data[label] = df_bin + + +########################################## +# 图 1:按可调窗口采样的时间序列波动 +########################################## +plt.figure(figsize=(10, 5)) + +for label, df in data.items(): + # 绘制选定的列 + plt.plot(df["bin_sec"], df[PLOT_COL], label=label) + +plt.xlabel("Time (sec)") +plt.ylabel(f"{PLOT_COL} ({BIN_SECONDS}s average)") +# plt.yscale("log") # 绘制 "rows" 等计数时,通常不使用对数坐标 +plt.title( + f"{PLOT_COL} Over Time ({BIN_SECONDS}-Second Avg, " + f"Skip {SKIP_SECONDS}s, First {MAX_SECONDS}s)" +) +plt.legend() +plt.grid(True, linestyle='--', alpha=0.7) +plt.tight_layout() +plt.savefig(f"{PLOT_COL}_over_time_variable_bin.png") +plt.close() + + +########################################## +# 图 2:CDF(同样使用平均窗口后的数据) +########################################## +plt.figure(figsize=(10, 5)) + +for label, df in data.items(): + # 对选定的列进行 CDF 计算 + vals = np.sort(df[PLOT_COL].dropna()) + y = np.linspace(0, 1, len(vals)) + plt.plot(vals, y, label=label) + +# plt.xscale("log") # 绘制 "rows" 等计数时,通常不使用对数坐标 +plt.xlabel(f"{PLOT_COL} ({BIN_SECONDS}s average)") +plt.ylabel("CDF") +plt.title( + f"{PLOT_COL} CDF Distribution ({BIN_SECONDS}-Second Sampled, Skip {SKIP_SECONDS}s)" +) +plt.legend() +plt.grid(True, linestyle='--', alpha=0.7) +plt.tight_layout() +plt.savefig(f"{PLOT_COL}_cdf_variable_bin.png") +plt.close() + +print(f"图已生成: {PLOT_COL}_over_time_variable_bin.png, {PLOT_COL}_cdf_variable_bin.png") \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index d7942a4..e254b54 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -40,8 +40,8 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.retina.mode", defaultValue = PixelsSinkDefaultConfig.SINK_RETINA_MODE) private RetinaServiceProxy.RetinaWriteMode retinaWriteMode; - @ConfigKey(value = "sink.retina.bucket", defaultValue = "1") - private int retinaBucketNum; + @ConfigKey(value = "sink.retina.client", defaultValue = "1") + private int retinaClientNum; @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) private TransactionMode transactionMode; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 2632ddc..6836285 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -20,6 +20,8 @@ import com.google.common.hash.Hashing; import com.google.protobuf.ByteString; import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; +import io.pixelsdb.pixels.common.node.BucketCache; +import io.pixelsdb.pixels.common.utils.RetinaUtils; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.index.IndexProto; import io.pixelsdb.pixels.sink.SinkProto; @@ -187,13 +189,7 @@ protected static int getBucketFromIndexKey(IndexProto.IndexKey indexKey) protected static int getBucketIdFromByteBuffer(ByteString byteString) { - PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); - int bucketNum = pixelsSinkConfig.getRetinaBucketNum(); - int hash = Math.abs(Hashing.murmur3_32_fixed() - .hashBytes(byteString.toByteArray()) - .asInt()); - - return hash % bucketNum; + return RetinaUtils.getBucketIdFromByteBuffer(byteString); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java index de26009..cdd0596 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java @@ -18,7 +18,9 @@ package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.node.BucketCache; import io.pixelsdb.pixels.common.retina.RetinaService; +import io.pixelsdb.pixels.common.utils.RetinaUtils; import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; @@ -41,12 +43,21 @@ public class RetinaServiceProxy private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); // private static final IndexService indexService = IndexService.Instance(); private final AtomicBoolean isClosed = new AtomicBoolean(false); - private final RetinaService retinaService = RetinaService.Instance(); + private final RetinaService retinaService; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private RetinaService.StreamHandler retinaStream = null; - public RetinaServiceProxy() + public RetinaServiceProxy(int bucketId) { + if(bucketId == -1) + { + this.retinaService = RetinaService.Instance(); + } else + { + this.retinaService = RetinaUtils.getRetinaServiceFromBucketId(bucketId); + } + + if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { retinaStream = retinaService.startUpdateStream(); @@ -70,7 +81,14 @@ public boolean writeTrans(String schemaName, List t } } else { - retinaStream.updateRecord(schemaName, tableUpdateData); + try + { + retinaStream.updateRecord(schemaName, tableUpdateData); + } catch (RetinaException e) + { + e.printStackTrace(); + return false; + } } return true; } @@ -90,7 +108,14 @@ public boolean writeTrans(String schemaName, List t return null; } else { - return retinaStream.updateRecord(schemaName, tableUpdateData); + try + { + return retinaStream.updateRecord(schemaName, tableUpdateData); + } catch (RetinaException e) + { + e.printStackTrace(); + } + return null; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 095a033..fc9ae76 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -47,7 +47,7 @@ public class TableCrossTxWriter extends TableWriter private final int bucketId; public TableCrossTxWriter(String t, int bucketId) { - super(t); + super(t, bucketId); flushBatchSize = config.getFlushBatchSize(); this.bucketId = bucketId; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java index 29fe27d..242da74 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java @@ -32,9 +32,9 @@ public class TableSingleTxWriter extends TableWriter @Getter private final Logger LOGGER = LoggerFactory.getLogger(TableSingleTxWriter.class); - public TableSingleTxWriter(String tableName) + public TableSingleTxWriter(String tableName, int bucketId) { - super(tableName); + super(tableName, bucketId); } /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index a2ea411..9b685ea 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -63,15 +63,15 @@ public abstract class TableWriter protected final SinkContextManager sinkContextManager; protected final String freshnessLevel; - protected TableWriter(String tableName) + protected TableWriter(String tableName, int bucketId) { this.config = PixelsSinkConfigFactory.getInstance(); this.tableName = tableName; - this.delegate = new RetinaServiceProxy(); this.flushInterval = config.getFlushIntervalMs(); this.flushRateLimiter = FlushRateLimiter.getInstance(); this.sinkContextManager = SinkContextManager.getInstance(); this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); + this.delegate = new RetinaServiceProxy(bucketId); } /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java index d0a7a1e..60b7963 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java @@ -17,9 +17,13 @@ package io.pixelsdb.pixels.sink.writer.retina; +import io.pixelsdb.pixels.common.node.BucketCache; +import io.pixelsdb.pixels.daemon.NodeProto; +import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import java.io.Writer; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -28,14 +32,16 @@ public class TableWriterProxy private final static TableWriterProxy INSTANCE = new TableWriterProxy(); private final TransactionMode transactionMode; + private final int retinaCliNum; + record WriterKey(long tableId, NodeProto.NodeInfo nodeInfo, int cliNo) { } - record TableKey(long tableId, int bucket) { } - private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); + private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); private TableWriterProxy() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); this.transactionMode = pixelsSinkConfig.getTransactionMode(); + this.retinaCliNum = pixelsSinkConfig.getRetinaClientNum(); } protected static TableWriterProxy getInstance() @@ -45,15 +51,17 @@ protected static TableWriterProxy getInstance() protected TableWriter getTableWriter(String tableName, long tableId, int bucket) { + int cliNo = bucket % retinaCliNum; // warn: we assume table id is less than INT.MAX - TableKey key = new TableKey(tableId, bucket); + WriterKey key = new WriterKey(tableId, BucketCache.getInstance().getRetinaNodeInfoByBucketId(bucket), cliNo); + return WRITER_REGISTRY.computeIfAbsent(key, t -> { switch (transactionMode) { case SINGLE -> { - return new TableSingleTxWriter(tableName); + return new TableSingleTxWriter(tableName, bucket); } case BATCH -> { diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java index fbd9962..d7f848c 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java @@ -188,7 +188,7 @@ public void testCheckingAccountInsertPerformance() throws String schemaName = "pixels_bench_sf1x"; String tableName = "savingaccount"; - RetinaServiceProxy writer = new RetinaServiceProxy(); + RetinaServiceProxy writer = new RetinaServiceProxy(-1); TransactionProxy manager = TransactionProxy.Instance(); // Step 1: Insert 10,000 records @@ -322,7 +322,7 @@ public void testCheckingAccountUpdatePerformance() throws List writers = new ArrayList<>(); for (int c = 0; c < clientCount; c++) { - writers.add(new RetinaServiceProxy()); + writers.add(new RetinaServiceProxy(-1)); } Random random = new Random(); From 21a0ddb4f436186cab58e9923ed1864ff7cc8e73 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 26 Nov 2025 17:26:35 +0800 Subject: [PATCH 29/53] docs: Overview --- docs/assets/TransactionCoordinator.png | Bin 0 -> 120118 bytes docs/assets/frame.png | Bin 0 -> 69976 bytes docs/overview.md | 180 +++++++++++++++++++++++++ 3 files changed, 180 insertions(+) create mode 100644 docs/assets/TransactionCoordinator.png create mode 100644 docs/assets/frame.png create mode 100644 docs/overview.md diff --git a/docs/assets/TransactionCoordinator.png b/docs/assets/TransactionCoordinator.png new file mode 100644 index 0000000000000000000000000000000000000000..8a8ed4b385792c63443b5e45442cf5c9a501c342 GIT binary patch literal 120118 zcmeGEWmJ{h7Y2%NMFnY8KtMoLKw3#@q;V6{-QCg+BHi81rdv{4>F)0C?&dCzp5Hm= z{~z~$xMSQg?uWZSy=-{jz1Cbap7}g;_w2=~D| zT|8KB5Xe`E5bsARTa~S8csZ%@M8=Erm7cDcqiH(2*{-Qe53?8eR=l<4DZ?YE_Q=!a z#)A{(%;n4i!T}THdKA8|qTjuI^z7s3Z!cd7p`#-||4I4r0j7J-w~qN6n|zJRX+iNZ zzL1$&$zHn?6~oMlxRoC&BMO(j!r;^YdQr6}2_gOcLo7oY(Vy$S;+H7*eqVo$e5jA` z=j~ETL7qRC$=V-r?>2GwqCy%#|9u6~lMnX%bIH~DjsC%(YXz+Tt4DK5KG~P6XZ-k@ ziUG+f3y&a8Bgv5FXu#m?E%B$tr7y>g5^klbHxHNB$}{!T(;*NwwBO6X<^OqYGym|G zGT{2lz%Tro^aQMw$jI1;+yEwDl&GZ5^$+b2t)VG zagU%pN>3T^3Pzk8#iq)5nX3z!dh-w=$sE6ptfzKqq9@>$la|h~S1As~Jt!5{tDUto z_M&gv4cxhWDQ%y!{JCCb_(#+5%m>qf98SMS!aO5y^rj>X9)BRH$D}qLwc5UdZh$3? zKZig*{5CbpR-f^-*zf>4O#FG>_jmjv(u1#tIbsxGl~b&etVF`Kq>^pCSq8@6NW@wU zDCG{%Zt(PRpWMjRuNCV^c^foJqo94r3pVtB-IKjGVcYH|(RgUySqTj1M`O5lKSNcD zuzLU}m-^RjfyttDcNz9YgjsBV3i+(!ZoqW1A*(I4nqG#ES&5yZBXoTYg@mQ;I4TmF z`ntO&tLNRR@pwWCyQ89jY9u4IIIg~xMszG{L%E~0x+S!KAce#OY4o!^=0hzkoDBY~ z>`0Tji#Oe4hZ}uf<)}p2H9*M6Hsk55}C?s*#$)=pOlN~lgNZ2qc!fo_1 zA_@aX{h13>z24$9hKAEjCGBIo!>;Y6B@4=@>SQDM?O95ZoM7Af3M>g!hn`DuE|4`W z{PyBwL*UYX`SRCC{M}m^t7-ZOWk%T;qWMlaRw6@v2LT#H568k%(zOlVNcvp}m-ksF z;a#i9hQOzDZ?b1D)z&O7NY+{uk-t-nI^d3Ww_0Mx?0secy!)1?$QSo&05OcLVoGo| z{lrnD&RL-93w!2{{KUNFo;VRX3wGIttP%!wT+-U7f-fkX0y$2m>!?#tPVA+LUes_? zAaDCPYOFhc8F`njesaKinqqw5fb!1`lE;Rp~OfYQ?6n4i)6I& zMy)20IgS?zMAFz^rx%SbbmFp%7Hl?fy-`q;hH5ZydS>0<*<7>hufS7@`b7g=>WU0R zOa7l|DGEDvAF2LeWecOfLC;%|jLe9e?9Ees#1Ldxi1_4MDw^(CXuMkQ_2d+9R2tEc z(kU(Jee#4pCpX3)p^BY-Xs6G%J8)Z?`xyK`WQH12M%A#WG{WzGe(kcvJd9NNnGdhe zhgzoj#pV+RI_eyTl```(Gl$s7;}zT7CydocThqsVHkp2D9_K4{&uK9EM|o37`FsXG zuzqs-$wu&|lB-Q-jX$O<`$0r+K@}CH;6$WBr2|xxLD#}4;4ug*-Q|X55k%Sg&j`)x zK^}NLb*!BXE4;bm*vyWKYrYL96E2d_QB8U;Rz0pTt=+EBB zSwo>UN<=cl)e+u1vvqP}zQwchHkp8(^#*-O>99_;&->+`gzF`SFHIoL>-FZeyo%{MwF^QK^F1d!npVFDNtXGO zpT{cUxshrbXy1Fk98+qDopOX5A#{zssCnw1ZSX?CemD&;BZuCdlq@5MSpHTFs(R;VYSP%%^>UD*WoI$(kXJod_#$x7SAK8ZrKn zV}@G(=4E=vX;pAAArIx#z^;JuEzwDbSRO2vV~)f9+OF50Bwo2pD#LHE*P2y*x_=On zXSr?ke(P=wJ!mku!&#wE8)IwnDv064er(U**geg^R=ke0Z*E;Wwi*fhF3rfd8KAN5 zrid=-^`66W@Xpo8Q2(llTyRKCXm#kleHiN0>TdEgQn)UG&tgZ6io29Y%N)%S!zyXq zEQ-=65gJKKevroU4eObZ`>pcSN*k$_1j~JAQK8%UR#i$K(!b_G#AvBr+NFcMtVr!I zet4w*$vvG3e(_1Bye=#M&^H?J2ti zzdd|M%%~usp`5DYlXWR<=CPl8?o?~^0U7T$l_8q(&0~DUhHjgPI=>J7$;12DL-o=D zr`ecsUw9P&pkd(ptN!=>ltWUQ%E}ly6H6gCK)#&r7eSVkthTou{u0d|!!6(sEK%B# zai#TI(((}zs@hYFffnG6>#{tA8)F|&V^v>3skZT&J$WiTcF;%-4 zW9pn5cV$>73`cZrjx?NDx?dY)=lhuhp4^zmxv=Aeo+@ssOo^(?y_jAm>}FujMfObM zb+5b*_xl(+RO?qk^i&ap_vDwp)ArDE>8Sh1rZZdOY_S(xRl@L+mD z_J&9arOX<3$!>=AzP5T@#AEfy8EW^s<1F;yb))I!)dr~U8o~mG5TPZp?Ad!Y&d0e6 z9x?J_(}O;P?;()0mw&Q?KknGjo0aJo(Uu1D0tojWl=)_xvEF=JZZ2>gud?#N4Wd^k z7$p2Atyrj!xe|hFD$PB`HR;FcSRfhEQ%a`{g->Y!iD>AmTw5U^`2|tAnwNqzp9OR% z!Cm1J#^bTrTb+i5X1?Fi=L>J+&OXpvxWQh+PyzAgrGhf&>q{} z9@)kN!&V{hI!IQ1;_*h$%8^PR!O5(hCsQmI#q{$dh~E7F4G?*UH2)vMr~j{cG#8!a zY8lZ$&(pgJM#gxmot5mnx>)u;XPfI>4m^%;4dZJ8D;~yS5!U^Li zIlZ>I-lWuj$ffxQ;>UuAAGc=Z2ix7c5!cKmCAbtgnSZRxM%dq(PV)>_8#>Zg5(Y;y@z-G+8@LzG zJCcMdp1wEGk{Lkfx3EAn)}+Nv!?Y5ReS3HVyIOP6H`&gw*@N|caKWXX&XO-Wpwuf< z9|&%ER`uMe-%7vvlKk7FsNE}z<>ks)R+(D2*aHJqbTT#J=U1?Wqg9gHy-01StjE;zb?fdn_#{4_ zbfT(n4~S6gECdvYGGr-4PX~39vM2GyjfduZb*)5vaA4uI=4|)B4gR$qFfh z32(&46K_@Q%!*T%>MY$1r1qiunr;gZY5J#_jpS)JBEBEG8y;R%`wptE`O%#*4t>Z( z`kZMheP$Jkf2KOI^|Z!UkI+koUlcvZ@y3VR4b#BHe(>WNV{9HArt7uH%14$441z~w zSMh5%Yl#A6qLzioc*iE4iX6U+N#NYd7p6X&WnvzesZxBa&zP3>p}-_{=znw}TVb1z zN$7}$`O$4j;MQ-4Gl6gA8qHU-HUN_^uj9hUpo%T5Ke?`4e10}!C-G{oo{Xk9<5ed$ z(N(GNm<{WQCnHt?r$toh{r{W~&vOr~tuobSzDIm1U6(<{PPb?b zCDQv)cYa$gRrSFRjfv>kzKx^&33yjbS2#WCxTh~?qH?kxQd1K}U@U5J+@^JNWLG-y zIKHp8kf75Z3O`VG`J7n^{JahIiiV|FT)HEt!{!>3;Xqr( zrO-dtpx{CvOx$$5bE-x*eoY@0!?O*otUSE6CzEQNR$N*6_2e~EX|mHZ?#V$LZ2rx& z@+T$;R1Vkplfq$EMUJaUxr60>It@-^$M8;OB)K^VHwxvaN$VtU3n_=D#;B%Jb7r2F zo&PLRBk~UZd}ft{jBm2TRy-{sedv>?lt|vs=Uv)!%Qaj2O7F9sbP(QjiJ(HkhwF#z zpQ{{}SJt6T`r%fO$$R?U0s>BBiAM(z#a{mb$E!Q086-+%Dp(D@;vq@#`HEG%UX z*)qmRK1peJnw>395U1fhA{^o^oDfSEwFpZi;$OyN!QIUGXuRSZtiD`*({b^D_#)I{ zMdkutMMX0h`*Q@jGnwM>a%+?<%FKHM3VnI!qx!Kq+-)~HZil+9_mdLkmxUCK0wvnl zo%A5~jO6X~S;Y9g$+ex%18;SnSfiy)N62Co;m}Jx?p&t`M|#@Sku5WSd)psKGM|Q- zlWW*5DZTKv7|y8cW@qY%9KV z9u*O?IIdHbiLor*s7~JK{6vE2FrV&mc5o5)u9wEc1OS$rYH0Yn7A&%)?`R4t}#dI#q_pjZBQ62GF-*7TgXT+U#W& zoi;bBw7V`@Rt7#fFBjn&>E)sVYvXT8467h6=Pe^(3c|~E+@JO zsmkCmNGDVdW>M=dM_`ls4Wd^$Pxztcc5@In-RpRAovYZ;+-5G%p&=DfJg%iLu)NGn zyURHcCDGEgAP`JwqKI?Bu9_dV(aUX{9ymH1hs;u*^Wi+3)*e@}V4YWnTFl}!3*F_J zie_%4W9t=*R`b3Xc`xgsThXOao?|nV!+Qtqx-{J3=Hc}l$344yLlplug26jzOBw>P zw1|m;S0_ji6T=O!pgg=`W>WE&krS(q!%YL|hk{Q787i}jUKQ1^tRyP5wne+U64Y@+ z#AB)QlVQv8ShRuPc*{F<2&9SgGqid11Hagz-ItH(L;EH}_-NlLV8a}E@9=P0!=#ww zO-OffivNwg^PJ0}@lbZ0$1^U)?C!_tSVPMf3&`^tOXrlZ6gpfA-wqGgH@6~Y#Zlz? z5g%24+K^h(TUT>%AM1)^&j&F#kld)ds&sND++GoQM>*kDy}l7k_wbA?@j@Va{AdnXvGSkEn2D(R*?nu|#r=3;Z0dB+eiwiiDu z$TU+el(_aWX}p&7$7LRlUHmF*^bR#afnK8J?Rry6r{D5!HAV(WKx{|uM%rC@eUbd3 zKdcauu2SY0KKq=PK_zizMCI@)eNb!&cAJyp<9R`S)cl-K8L;9ufsUe86WhyR!Zeng z^gAP`V?qM|Rq_LgN`KCpN-2^|AYZF^KeE25Y^bNeL)FZ4!-q8bG+`>QrL>ctzg>jGm@USW<0ug)BiTr|rlr8sPA0HcA zV%_|r%$XDX2X!qxANmm2i7@FyR$v&XUzZ<1Ja! zD=!7f&Y10AD?`DKS>L-2P_)IeVyF%;6v+X8A}s5{)1b9EN>&~-({i{mm2N;}bjogb z#KHWf8i#gF6Oq!`{QBdPGd{!V)tSMvM@S^I07_Md_jiz0h?FAsT9lX=*_yiKaF{=S zh#j5HS%?C~tmEXSM5yq6xXxe{gUqva)r@w>n|3YwHKan|8m>LmTd@2r|9HV!zA>}Y z*|(aj`+=mE=qucOD?DM@pVyO2t{2?r0OyXGkT6n zrPzZ~@!AqYE-tLAh-w-;;rlhU=R?j8P5d#ru6cgb%iZXAAg}2DZw{KC=gxFL5&ua% zfTgDRCk_GMocFsgh5Uav65Ubw|L4Zhs~RG!$JOcujJlfaKOT8sBzz@N^*oSMGV_W& z9|Ec7{^ua{Qsr<{{bT?C-v%e};n6>6oa?Cv{ez$H|8*|0Z^4lTy8h`{K)2Ax3zqi< z`JaPyB|+3j_^Nq_tMv{nS(KAFoD^aLUNB22pg{Uw5uyC+_gMnX&CL`td1n`!g{+oq zeL2#(_uw8q!@@e4aCdi4WOtxeC>1Pi6{xaU6*Vx}2q%+_Pe@3&vN}53*CZt+jm!@w zV7GTX>}FbQ4U)|r9uP6cA|T)}UlOcUp#eM2ViM#@_5s_Tdwyq@jn2sm)5|yy50CVW z3=&e(-N_0u>Zn7MtLtmUG6M#^zL))?^_>gNeyss`q*VFUrKR*$YBV%7RW(0jV~@ONy*5N2}BR>Eh7q^~KQX+E((z%xr83X|)7L3zXOICvrHEPdtX) z$wvQjl;iZYH1~;(p|i}48({BjyW5LnLluYXi{qi8p?;V^Zvv}qX_)>*seS^BmBC;} z_ttm`I23_uU0vOh9K+FqGULgURzjB=+kFisrK!~<7Xm^;ni-psJb60Zo@Y^OmQ|eB z>tdW&dpzdm#f615_%Q{%HLjQ`DJl6m$0+N4DLkt^3DuVC3t&gh*s4cs{cgRFiuDIc zCT=p5oF^(w36&ouL`BTjySQ}6GKh+bW=pHo*#1KMV7)bFy*<%ir=p@V>v)LbWLBjU zHhlS6N@}P?w|BShvS)D6Vy!Plsu^uoF&Kqz-ty5D<3e+?-nYFeyK4yz8 z0XTG?9uKv~iqr*y2$oqxap-ge2ZG1O#>$N+>+FwxBTI^l>)t;#RcrP@!d}~AW@fJ8 zO_vaFub6d)&icu~U@(0KPR^uZ7G`ED)w+19*-8uD!VSll;`o|A%=p*Fka~m5Sveh=bbuPEg3Voa4+_8fY zaab(ZQh2=1a@wboxI;aER9Cy4jO)Uq5yZb!zwf7Ml!O+MPJ~9tu|y1FSc-euh=G*U zOXmFZOBUJ7lkJ4~__d25Le7HEZ_ffZM+)i{CMw6-A%+fpgTbnN34a&TWVlOU0r>FTvb)I&VV+Ph)$;qJt|MNKJjE@v~ae@?odNh zGk_oVV}ylKf}icU1$b80(m*=Y=y*6+w$%RoK)cSBJLQM{@dn6{AR#&&t?keKf|IG! z>5fG^5sG28*$F1%IU`qIvYB;ijWn!6qYOD$>%zdm_$UorSu#WTF#^KR=Qv-n0Qvp< zs9=%wZ+6woFT3_B0qL`Sm`T=Q=*V~9&XyvKk-~w>va6tAZ#+!*bT@u)%yDg0dZ4O` z^}ExcsRapHw~K4d7cIhsr~>&aaeVfUs_z#2{&Tg7p$FcY@H7w4gb=r(xiY(QAC%1- zr*-`uEH57oKiZnG|8ni%!ei~9I;GnX;I!vpO34MaINMX_@O6*pbWwRi9o-cc`ibP= zY{8F)(RiXswl#>bx7xvSbHsR*8v4fm^2C@W_6$ZQuHTwvAS{FE_&$1J8^bG{eHERs{fxwI6DVcAuioBkAKfPrxO$}g>N zM}^HP`~`+N^WFpH#&&!Y%f@xd8~$j!5E;BTC}g$=-qb707B#W=l2NmkIraLIIh-zd z1q8f2JVavYeQ-DM7>(4*hSGEoeOOpog9zATvCbfo^Nn6KNU;cW3hz zD>~4Qrx%eQ@#yG~C_jnAj3`Ko97oQejq(2^8vo%J)6GV1A*0EZUJh=0vGcY4M549P zXu_`ACMQ8*=_8K zp#v2Ux|&{CX@6`$(Dyumf5LEY?iUdek!81Jrf7POzdy1wIz&%A*z*r`F%aVU5#ME* z!XAGqtgufsj+0%MLG+P>Rn*AkRG>acQsK&cbIT{uPgf-G`*_y3^r6PqY zwONrJOr|rG%(&xqPOOs@#KazfedWfAdd3nuZ2G1xZIqSx@8jnhV!Wwdij z*Fk9|>39aisUv7urcGPd+7xDZsKiFM^c5RVgy-3^cIoJ#gSRa@PA zxfjMeT8tbXgsoaT?%eR|%UdPdBkorMk^6i6ZbTPD;)R%p<=t`Sj+6XmI+g>P7N3^R zTjiPb-QWs3ZKz_ji;es0L}4i2z@MN}EU+YE#JqE9*M)!%JLh7rcyAu~H z%SctGmOSO%2@Un(qPext<|=HuLqWFmz#kWw+=IgnXvv*lT|@1Q*fo>?vfrfcY=@|N7)`+nw^kR~1#b+yRuo!wJTti0yroZ5v*Kiv6#^)Or zVQ3rMET!EMer@7vN?G64>5Trny1?lmws0Uq`tD|#<4#XAom505(Zv>JB2-5ig|n5` z1F^CXuEv`GwQ>A2Yf+^Z?%>%KVU`~=%Xt;>#=*G8=YDJJ%8KmRy=igQWOHN3V{da( zGk@lAJJqg->P475WJgMpMk>Q)$gO%}>06FGvnhnO16)cWV^=%JzPPo*xW?rgmt7tU z(moqmTnkBl|HFD)0i?5m!NEI-O_k^%u`*jts6|Mc^fgR=cP8Lr-`QV{i{fB#jn{Zh zb{GB^VjRYA(S{trplq`JFP8-QW#DLY}3Zs%@rb0c2Z3%bVSx7HcDE$&h4P*;9ftbc;pC0tbeWdSQ2T46@!siS#~Hb> zbIf;>t0LMZ?c|tlWsz6s9B6H7pX1WJjEzXwg380WQ=;SCBh1R|_b>A2?eCDP4u5B2$)sl2sVN4N=M zpWNSe#1`|gqbJmzULr0YctS1|h@hQ2B$Y9|wV9n^ppTG}VRb~Np2NAbgvum@Yfkpl zbaY_69fRnm-Z**sv_M{BZz;sqZQ}`d?;~52_IBafO1+l!ubLq@UK%}TgjP>USkk2{ za273bQ6@vJtZnUS{31aBAVtXz4-#!lG>wcu%>{Ne+VuRaYAQ5eR|ahi+=+U|@gs() zj_y%yqRiqq4<@~i?vFzDUCpP~MdPLxq^&X@l)`%)z1xQw2V_R;ah{j>YMqwV-yCOZ z)m=5|CP-8e5SizeIV`R~DljFMSJ@Aa2o;n*(i-0Y_EgYb^g%UM!Y{R1t#*s9*Gvzn zRe&VVEQtVgLBIvMd8q1L??HZQ1#;V>n%}6ai>H&72-dCmrYCuai9w8C zPQxefgI03xd3gpApIvV@d%VS>a+J}@$Z&PGY%`vgZVi;DN%?VkZ&Hf`{X1^lg-Sl9 zCuUtyClj?sqTob_JS()F0|Z@WgW;-G>)SH%;jE2C5LpGfos*>fvNjfc8WnX!<`j$N z)#PU*yM8=ZSaFlzQG%vQmmSnHQJz&6=FTv=2MX(kKhsgg&hTH>^d8&FG+TA2U)tW( za2Dfe2=w6eTcv&Y$DGYHr9JKTV}x;mp*1BA|JL?k5qRiKYXiVRn5 zlL&IXndgfj*(WU#ocYBHaYdlN6E$Bnj?ky2Wd%<(p5|txzKfY%>}fd3?*FtBE}g+ci6KhMTVOI9W8_|dG~=tmJ<;fBbhzzOhw+6Tya7^3xKIO zY2aF^!YR(8HB}5>R1|@Q*<=lwlobV8PriYkzmFAWw!y}%p!afTMh=-c09{7RSuwY! zc;8O#e4)ZC1W$Eq)30edRFD?G+Ik_41bicYY2@7p66)@@TFG?h3hxEslc@NkF-qs= z-4GE&TbE#VYNUoBg$o~V>D?iosE!q$ZhveEqRyAG73nh+*nQ;edI|wZ9JK)aIi__D z?^#GmD=0Og-OXh7*~WL-x1|QRr|gHmb%%dtG~^#9#wAg(SBU?WAL))|8KOc7q2Icm zQU=i4k7L;ebSCNrea&>^U!pHpCN#liV?{ra#5_ZI%$$*ohF?6aRUlG!!A44m620q064riiD_?k9Fq8VeZ?>hP zT{tcmb>Ce{v)CBd!uwpCDcf}6olZ-m+kk_wL(lV6n)uiB7z~*Dc{8ob9L8S={hVK| z3*>MdwrTv7_!fV>Muv7f*Q4L|OD)p$_GJ6MOGQPD=lO!f9fI-ZDWwJ+g zrM~_wEt-L@(B3(NI4x9e+EJI-v677WDuT5l4IUntR8Oy@aZ&zR@yaKi;}^Uyyol0DyY^8BYV(-|1NuAhU{XG}}q0=8-ndY~% z&iPVp>(G69(KUGh2)=EH;^K=|Ls$A9-r4PoV~-T0xL`ivgIE8noPP}qe{FHA=kV-} zFq+Is6Up?La#YPU!C2HZ!OAN+YZeFxghT805@lSm?f3OtM znZ63AhwwL@2G&l0?@FY~fsR0?mx1HL4Xp4^7X5pLxYbp4LKUKS+sXo*<^PyWTc;!N zfz_x zKL%v|&7P~NZaGz*IK>ofCw0A24-O0y`+8Ii*&zH)T57<9j|ZYnTFS2%f5FjU&#`;IKdyR_X2j~tOIl{Bwa9OQM17zC^+71+ zZT`-F#j{xhvn74WFV5|iZ`5nK*TU|s_Y&d-mUYL*Zb9@BK0HF=SexB}2?Ut{4?Bhx zj)M;T+qZ9N_1?ecZC@U!qCVzD_u$g}`uouD7Lq0V@Zm!S;+I=VTd?jm!kvM7+xrkM zM z{KIVj^;puQUDGSr3&_u3W5F@W$uEVFyPe@Q=|$YyD59!K9~ai-XL7mMetcx%!Sxz+ zLwAh$-F-!@W~kpDJb)ypND-#CIg>&w;Jnma5uCR0yyIQu+E7`Kcz+y)Ls0JhCd9wP zr4gwK9QB&aNXKEJe*=#B1Z<}>l8T2!VphwbyX-2}p%9vmsk2p9Z5=O*DULwT{kMMr z*Q`!<(e=AM@gBc~KyZoLcMXu^zZZVW-x*k}Vfdcyae%0=jLBN3r_+rI)eRe;-yax+ z(7*WGb%H&A?h4!{w#?$)gT(rREsg}C)t7DyW)JhR_ryJUg7;-Lq;$t=0lRyv+g6T! zrD&OsSt>dvf`hb#;M2bX^Ukm8B@GP7{kZvq2&nSmAYb`?YWI*_y=#l!YAsYo!{OM> zk`FKBY5eA?Ie|&Zy(#N|`@GAQB0=694 zuDm!=N-}E7`oJnwWmL%vIp=2 zc6ayH7wn`tpL@@VRQ0+4$vf7-(>YM_Ub`g@Lm=G{ymRk;S}TdPtRuln20BU(*j_|- z4zw-nXV!33;k1*qgkcoAPmTHc)|YvP_rt5w&2g3~`hVfb-|^+>uH6}#q6riQi*Kr1 zRuU4~ES8HmseVE+9)kFhN#)x^p)$4hRyaouB$ce=k$%+4l zz;wVP185R=YR;YWewKASkh*$|0ZAVJxobU1R#xQ~WWMT7TBLGka40McTR>T)-9>(s znrOI^RJ&N5?#hi>@;{3j!t{)SgU~~bxU=e>_OmyL}jUJ)?ed?L@t>O81Vpd)(|iM0hx1!PQmaY~eIgsnTF2}o;qsT4fw z-Cg$3e>jvNk-arj%eraUSO!!D4i1$neHgvt*`BvIf_}EVnc05wN4H0)FW!d9&#=h5 ztXYdBu#^EN2oRAM2c0zO!^b~+dkX;{zYq#Ul>wNrm>4#;bNt@kUdU^KjbnTLhhKU2 zqB;dB6xi~A1xqU};6{~i9F*~4hmEmXc1jtnTc3bZ8T-O*%Z^X&_5k0dg`*=YTk6#` z0%YrfzBm3$6jv5?DrC;`V_ zCk{x`&CBX|4yRaAdd(L9qM~;|icuKR+nQBX{zf1Vw4P>S%HgUn^WT2#Edq|~!-^tg z$US}LIpymRMG{pfW~)znTQ-@muy{!(v>T*00Y5j`Q03kyPpWKQcr-s(3wIxa&H5kBdf0R|4YPLCBe_Nmo7sh_W3#F|Dhs zD?kAk4jatQcHEroOiu#Y!b}LnB|pFCk0sbcc=*96+UFbTi|k*i#x8zAT3CBSmU zawB;PM1z&(LX_*E3_;Ba=j*bpFI|+#=39ndL9d549lN=A)fylRHetuc){4+}& zmNnku=5}yk&e4{Y@l-yWoQmoiaJPU)>5XSDF<<_qpz`AjQEsYft=S?!`T9iJW~u(* zN_U(IG_=)a;D4~BV9+#699nfnzX!TsD4We!7WEQs%?#J+8=xwEM~7ngxfu?2ZP>_G5kM=qq&Rwb5d(7P@%z6Gh2j$n$2OyD3fzP2Pkq9nu(0bi+9SiC0C^VXPDE~**@^mJFyO!LqS?w9WdV$ zP!PcB&`3c%U|NR-0c) zK%=PBmq9Y{InCcWKU^&xU+#=b$CMG%O5f;7VBPX$Wo1qJV5YIqyt^46~EU#KdLXqDRUKBIp@`nvAIxl zVi}`NVLUll?i{Bry8^D>6$wjC)hoPHC_hoTa3H%g9GFg5Y15ttCByc64#^{YbMD{2 zKW1PByy_ANNKM2*a4Us!bKh4y-rrA=9+{X0)H+3f4c@HN=^b-EyYD8Nz%mRX5v`|C z9Zb38$h91GGtEj#Nkw;^ssl?H(HUS-tJFkO;URkpg^}zhyH}bmKH83pifV=CvH8gs zs+1Xynk;uj7%`BM`II|eoxN8~SM1LeBP^%9lTXB9{NJ_HCC=MPllBzpb-JFj+~4fy z!JbWrVmqKrx?&+odc7mF<5ML>Gpt`-GLL{TAC?%qyI6}(G$_}SH%GcAc`1tsaUOGBCjT;CFJoVniV883BP}hh?_gs%*Ul7)*5&3(e=uWz7!7FkVM}N65g=t~ zB+%saT1m*u3oS3Gf3ge)>Cu<*`_(AX4x{$6JW%%%b4+IIx@BI!dZibEHkNV;tvWIi_z~K-DP>cA4&91 z;8u?PChO+o%lm3JzhTc9f@Ic6uE}&&B^a52Dk&+|UGA`&Jq}Aq5J7uINhw^@F%S%< z1o~)QvZ)Mb4u{a3hb1Yb>Ba%t3x8C8%ViOuWTz7uqWCIj795 z+ssHY3cmpj3NF>n#dACuI)QhQO?R8qtwQ3wRi%gw^w?jPVw zk!CiJ1G}TX17%KO#GtH1r#n^Jz+uopO)dF{ayJX5mbP|G*VB7OvvmnJ5Oo;c!oq^^ z3g>-E!=z&kK`0dLA3upk4a_|NXrSJi$H>SSNhK>K)rS!XR4_wgP<{rhZA{G`J18P1 zA!q_;x-yW>lOqaruBoYEvD#>o<@81|SD$8y?(6%Eg($}?o-bX)Z5_Lrr-}U)XLHY) zGmYpDiyIAv-kNPeKz@#(*8`zL9pe#>N2enUYA>j}{DCjdhhZd|&zM}uNuL`O*Ai9N=->nQoE#nH&OstsPXlGX#_{|B$cNxry_vWvulwjU zIw98mw2P)5{0>_(J!8qx&BX!I#OfNRYf(6UA`&nZ^j02lKmmY#ft-A!Z$b_B{2K7i zzeWX&!$4<>z&H{qDnir2;o`c9rQ&&3kDPU!q^F$3Tp>1oJqSS2x#}wFNmP*RCM}lW%LI;yceyZ(&u(8Tmc)_YHJE4b;9#>m zqf8R^eV+RX<#6!5e}rZ|drALHxnAii$)^j5Kx@IzeMK_8kOIX3dQN0Wr?bh*o zt)uYWYq!4Bri!wUCrb7hkk6p2{VUW#rG+Ff>$T`3B)wxGt`{sAdHZ*FOP}+$>8=n@ z?AnEyD06~7faCuE@+jaR%;`JndD+wrzNbp$GJ(6sG9Mzn4!gf2H{&&J3(WDT2^JjW z{O8sU#6Knl6d^>9IaU?-H$Cu-QQaWsldVI6M6REGJQMMold=5xM=nVwR-ms;*{kH< zItCe?)6+i@v`cTh#Gj0*+KWIwqx^Rvne^^fk8v=cm%XHYPkgKGAtT_S;6_ATulJjn z!BhXsCMW_slN0aZv~pZ*(J+*qYXBoS=k#~+yqiGm>^B-fzd@Y_)Idr9dlHLhq0&ml z@8|7d0o=l1IrBwz^$Tah-6y&mPS6Tb-pwb!6MhfoO|3i1^Dev0HLEwNcE zrX~_LiE#vcNKd$803+giFaqi+i7{4RH~U(0(5wCnwi6G;z~fKw1A+%q`a*;qC7VN? zypTMYjFChu&qu^bg8b=r{~EBSr$#WlZiu@ZeCKmodxajqKq3eu*S}pnkR-uV3p1kY zu^75obO&bJvwz%^+SDCH2;9Kbp~aV%4Gtb^h|&AXB*NO^3d%!^HDpBJ@7WLt<=kl=eS*)mdO>i>ztou3=LMSz;Ultj zL;fl7U!!J(-WfITM>uy<_|>aQh1dB7zTAKpVGcH{cwaA8!2+IS3!XGgRETX<V=>A>2TPqNKbT{KbRdEfWdJo|JMqzzfM%# zRXmqQHTMQp`TG>@dDV}j?RF-lWz2!948Q`?i`K95(E*C`EB^n_iiLUd>Um`I04W$9 z{GEjHSSVXqVU+hS$3d@BaV7=2Un1TsqWSNV-SxeXZPtAM<9YdpEWqP$f5_f4`cd;g?WAL_v zad7G|jNm?m1M9!HCjP#)eJEDahCuXjfHB}vAA_x7#5y=f%8Q@gq}Zc2s1iyBGOgbq z!~5^Op7c$T;9~)w2NgS5;eW%+))6tn@DYT1c9y=1bd3=9|GJJpVd9o~$M{(^JiRlI z@ZS`k!>dhK30-5Ekc>Nij40Gj+Da(loR(1KQ%arQa4ggAkDmf?N^dZfg~w#F*AQDU50HjV6jeEBT+=0l>V*M~z|VSc z00x4=#AmJJx%tnp_wFjS04r#V0Ks=;f9(nf&+GO5IlDdm?31m)X)S&I>ym5ys`wE# zK?iX*I?4iOzQ14)J&sl{#r?~uI8_+5bHD&nMEK8caorVVtQm|S0JxXkF*}=7*8m8jCbVMR-b65Y*T;Q(a}6;1FGMgP2M89T3`@() z_?*sDot?P={qT4q@6$q|P>1sa;%#nKuZS;z1Y=1s8Ojn*ayp4oqRxC>G^KTZuw1Gn zBZGy5BNW&F$PzY24bbu8>ceWy+W_<*Dz)}L zP{14O}Da>dQSVxt9qWhfAw56Y%nw;JgBg0!`?-jrPl_&?tVSo(XU zV?G#t|HzuiW)~Xu;L#(A#|+2A24!Ae0-z0&$&*VN-Q;5(hP~HpalG8F00d~bj8ux> z`gC>0kEZ{Ft*;J;lX%Im=q#KkRx9+0vHQ;zd;eu<^ran0i4v4 zd`;j_eScK^8t^oLHuN}Hn{tyB6#;_`iV6zNE@0Jy0Eoitu z*nSJz-r1R!vJCvDyOAhQFqdz>xVZQL%8iY@b@1@XlR6M!fG)WQUb!Qh2{=Jn-aCL; zpjjpVxx9STgezoVV9@7~k)HknW3weP0OhPz=Upiea4ISQrdvx(%K?R`_qRp-6|$$x zIy}!0R2`|AM!$ss4q$D_>5`u1;ls}%z{US8p|%( zqwmRVESFen|Gk>%dH0jgA=`aK`Y++Xz_k;;jvW~NBTLrF?+nVXWm8D1{*hHyc1rYL zO$5~}e9{{p!uC8raR4NA_U_sDkM6Q#8$^JuSnf^9j0Bvi4qzi1_P3@qm$OtIK7RPn z+|ptJB6P1;D3eLO%q+Tc1PpirR_bRY$ipS1e4ZD_%OFbnu6a;$OD!f?)u+C#S`utalMPES_D2WH> zD*ggtvk(3Yc)9Z8 zzG?|1vNzkkQPgmTKy}8?G%@x%i&v$8_%Z?;GH|I=@!F=h3n-&0gQhcNO9ph(&V})S z2N>O_dOYPKNmgvqPfhHiR&~p=HJJ3LsY*~#kZC7~{vuLMljoAt`(V}u7~b8_4@LnA zo+D@v2NZ$hs0ml3?bjMtSy3SHF!(ilipAhjE1LIAdT=oar2g}Tefw5hV64B-;0NC? zR^vZH`xrW?#;9)ux!nTTt^uPZ9(-5NP3ZOreP#wftKMD?F%i)&=w}HYz7Gjmq`Tcz zjB&Vi7t9kt0bPu5LG62|wzmZo48imLK|qHT-6jKRPOHw z;7cS9sAA$ZOm{FPdB?QRzx;zQ#1pHW52xp=9Wfxk5pWeg27cIdf?mZjkUDYT_j_}} z$f}*27x0&$u90~DUzqVp&*Lkp%;p)OyeS0Tk)ZWga$D*mwaF*SlVt&ZKyWsd5NF4B zI*qr&hK4LF(gp?;baiI}1D(#pLC~Lqoc?~<+QG(#l;aHbCd%Yw-q7{5ZfrDwnUm{W4tfiuJBoteqNTII4wPbmhPesv`8&eNlr(*_3DW+e$H87=gK?-Z~x zUYt2?5pD*fNuc^>VknY+Q6V{aY$YYJ@jeetDq*k}HFC#wC#_m5`l}XEMROMZw37~M z6c7xQp~Sx52~D|a6Avoz^*2@MY?)LWfi;D|DW2NuG{QVV`4GrnYUvhY!AzAB@-pmcoyWTlqq6XB> zR@`Es3yqtDp|#Sp`7j|CwXbiZXX@N4KQ00Iq}K#f zR#Li)jS{VHQwa*Nvq#8{12&Dd;Ve>?$9$2j62P3>_XJ`Ki(N#KH6grw_)}s~bM%y=Vch3(0k0&@Th;nyB8O(J8g&d{T+z{8?XaKm{S>v5qmY%jw zRRD(Up{XrE)jn)FozRLSRTv-nu;BpT#2-TZ8&u1$a$vLrxZ7-6CZ?eLrR8PRa%@l6 zx%=R{?#x~-tNT&^<}eV<8m%9kjE$*1j1%&z4-1obc6K%~F^OT(CT7ukFxB8WS?gj4 zV8j%VRVKtDsQ$AID{vvlC=w{nd*-V-+g%vo zq8qTB0C)nEl+$%?d!r}er$^gfyDcR4dbJBcO99#r!BN^f*T&?a(*3lrEReKoX+WOM zSZFv4V1b~T{=?7CMrvAE?|(J@?@a9En690yViL&0!j9#K!SZ~2dEs@qbzLCv^xG-Y z4QjPz8>5Awb!w+`sz*}jS|?7DPcS+LM%g7I2_|e3AG0(+=I-Xci_J22YvRtkRdkH- z8$gjc@OBlZ#2Dg#c2n86y0osh=XB;gNG~>8rKF}LEn@#vU=JB#B+i7P5_^td&4Lg- z#km^sLiCkoRXsNNEFP~{*COKMJE&l@fwcpkv+vL>-H5itbo_a^xORFcZQ3XS8OIq% z+!DlcMRgxd*m=ZGRO38!AU^YF@w^6Ktr}bOAZPXuEv2fOTFZ!ZJYwjLC2#%3q5T=n z!jvOEzpq)4E4IQo5m(I61AIY`_SG4wBf1hEZcJc1x|n$6DuF&+II{I9cS2D|=##ow zllKeJr$&xZQZZ*tgMRt zV?b_|ozxNGFbxEm-y)tGsLF!eXE3))JshgY_n@oqib#DSdS&4q8THj6JUk5OjVrHq zcM4#W+_YlGJBAYI>=``CSvC9OhsV!iMbyB9@zYj^GX?c51&k`ld`AWR$*$eU9m2!- z1bZt1R^dvJeUgRwM-CJ8lqaZmOH~viPwgdJx&Q;-WZgKXdG5m5J^F z8F5jZtKn#70XM*c&<+8vuB9oBOvK;!3?j&LJOG{X_r<#v1T8Faa?yAxvp3|}6NG1m z4sT+AzO;Y6Ys_$DYh9kj?|1wqI&Q;PrUXG9D z1|vqqI(x+bvv9A?0v1F?(6O47_HkHbdvLQ&)w>vdpW$4+EJ;v;U1MB^zW|?zc>3o? zR~AAp?h-WeDN0O*MBunAN%Vi1XJ&&=(pY#3exohj+u%fA`rfREXz&{XRm;$uGZBDDz9IE z-g#dW(63AQT~Y`KBlTo>Jf9AlfW?Hy#Wz>>4D8=ooBaPgQa60LSHu+&ZnX>=*v1mJ z6iRD?WbI}x^MFg(Khb68U?jP|Q@Jz#8m#BVd?qi)oxj%SEi2ffxfh~0kf`mY)!uTa zhm2B*iUbJfds;nf3f%u0|4t9Q0XBGpX0QK!0}vmLsW5bFXu6%qft+WHm3|30S+rCH zBEhR6Z8@4BzYyQ|LkIs#2y9X7X9vd;ac5cTzn8lnEQ_Bg4*n-7?W*na;4ZIuvD7&` ztuLEe-jB^EOapWcP91-u{DH&0jPu%I{6>6$1=|32N{LWC}M!)7io zj>14k9vh1UwvNk{=RzEJvCqhVuX0vKYEg&_5o^YsKC;&(6Mjw--y7M8Jba|1_C(Dz z!bOa{$*v7>REV4D-`ijRzkmM~JrH^qs$ME6h|Fl0nGv~EeERg(3)c+&0fkw>$eZ_X zDiH>6R9+blI+TTu;LXV@d}Wgv>>C@2G)u=j0?!vI?a8 zc7Obwb54XK0oh80y;a(vk7hYm&A`&!Bu?6ir~35QqIc19aqXiCA+NeL8I_zb@ig$- zE}*-VQ}Edb{C>{oe2D$^DWT?Ndfn<)UFKnVXu59xl!n5d&i9rco3b6U$=hJ@^*H0m zMfbDi$7Q1$r7zz7l_}Vw>fDY5_xnT_LO|;vxZ6xq3FDjp@iLRJs zb6R{_4wSV6?0@+Z-(~a_TX?3_n>UaC`OAu;u+o#JXxYhMZ_Z?a z^D?2!IDX>S`3eBpy1w||{97z1^LPG@m9(|xaIv)Z5hfrYfY$&qduhgp>>6f;gqcL4 zR82|VVRzyl>JLYJh24ELQrk1F$|=n;;pRK~$_N}D8c7SH80*q-yWK6UwLVHW6{>lQt2!gK z-!!uoT|!wDo7i0L`F8OK@0E<)TkHGtFJ5VY7sZ+|2@*U%8d+a9)MW7mU-(A}n-zC* zI+A7ulRNDM2wE3_gxYJoe;vy^3ZglMxR>h}0o$->(f z433k30TzuaM2d4p(0cyMkhc|8odx8c>Wok_@)SvN5jQetgD{!oqwE|EtS7aHdHJ~2 zkK%zcr$eO?q2qh!CrFYzp;PG`I%8&V5A~%*k;i{;-LJ%}D1!6Deyg!z??G=jscio=F0w?Y<-B9=?cJOqpDvLm8aqd766}mKDa@`b7=76UMy{Pku zqfZ-%L>GWiK9n)@B#H4@d9D9)^k8by;j)%tvV$#`GqAx1$;B%g8oBk(^tq-0 zX~Mx^6k}sa51a5k?ytp?&U)Mc-TkNJzP4;9*a7&@xw%M7qFubHYS^w!3mq)8GwTcBdDCFi^@;5(~{KNlB)cM~D>cF|`R=o3BqlhL zOpkI%x9b;3FcIj)hGasUUlkGm0uth$9=@aBm;iE!EUS2ta-4YJT1B9M!oBxN{zb{N zSctP9U*CqO+?->KU4|=u7{METcdwUWQHUbq1iWC{%T@VigZ%b~I1lolSf{dAubQd% zUa6|m*Ba)`Chc2+Il$QKUK(;~zPT6Ha-XkFIV`CMDf4v2DYRv?q3>0Li(qQmJe3m+ zL**sWMrI9rq5C+cBd(SSav*spmN*Q{?>Xv)m+bdhq!|O=RC=G=e*t*n^_aNkel{x3 z|7T41xt>HBU5)* z@2Wb;X1UQreKgKvrzLdu%)5!VbJ}X}*TdBAW_-4?HnsJXki}96@C7}ZiajBz{v272 zI!!IN)Bim*aU7ql!Nn+J^=`f8k@1BaLC!m313CMV&(j7#ZU7jRHSbWNH{Dyu6fSCj zQ^Y3o9~2%5D%;TtJ%Ex2fl??;{#NVXCh2#W-lll`_43<0!7}d8a~&n`U3%%3=t>>A zal?{KEp=1uE^slbSW}q!G^T#8j%V$~z{v(Uvt1g#+CDdBRtQ*lA^HucM$*tyAVc0` zz_PjwKU(B(1cL;DdMC8tbH~^wU&$AbHymG!SKG%nu9J|XaaBX|F9zVw;snPsC{t#X)tlPj;zQ0v4 zee6!-X8;dx{)@>2!0k_mO;i_OQ&Zo05rBo3dm;K@RG8X3SBrEtUWo@gG(&q6GuPsY zeUXiw-r~t5LYb6KI;z*EEpY#}E#lm}_Xb5Z@@Bh1F+^ZhIEIiFAi>vM<~=)3M%j*~U^2uL&&h0rXl zrT24v=EOETxK5_spwUUWFeN;^nVD+&%d7ko8ZhV{>`tiLzKDSY$ znCOFPJe}*3sjzPrZ|K*{9&N6o<(<=YoQeAUjZoNf;cz(rb}i?_hk*jh;I0y64617| z%7NU`ugDEv+RvRDyiE}2*WH}uMygz~DIWcWrhAgiwDxOv_)#Fs9KJ^@74v3xp2bNm zM@2?H1a#P$X-?`=!ZLO>vjh?VH(M6{O0WyMn&Xkt#|*cq{>t*-^72&7t*o|JRvx(I ze29tJV+C-&v8oXA;k%gyRf51vuVZ<}o0hlUhmSHl4$`J=6@_qVoJu{SG!9uz2qB_c zQ=}R-KI>Q8hgBU{T6`6syR2T<5PgKL*w0hR#VeGa0a~XH&@5*>Rtx|-LSRTH=*0eP zyQzUkI!QozZEX$7@DhA^^AO$$uY{OBhjG~fI@ zQEGGaXK3$fxyn4lO%PdDy}-j8LbWyj=D-kX07Mbw3>649h#x=XFZk*apYm9`kp4H(uvRi?~%pJE-JzY zY!zjgSwI<+Cb>(Y*`;pwZBH}mg-WWQBM6BQe~2~8a*Dl zvHV+z0deGmj@IFD&oW2pm&5DoUy|m{=45B(vmiln+CpzEd9y=YUd$!@YIOg=9N^a@ zMAz2WkAH2BKn^67FKrVHfWO4CnKyVt(BXRMfgCi9Q$? z+b8WM>-BoI!5B|boyU^^m#neu;~p^B?BUT3 zWQUPsjgEDb%?%%4|NlTkol(3s6+%Xr&T-X^jQ5qc`Lri_B&9_{1DS%tOz**mW*>g4 zqH{O9-~R2U(MF^$S|sTFScQ-~Bsyx8zi1P{e5#|m`2&p%!98C6e)7HlVMKp55QMs- zDu#DhF+zE#hUjabCP-+15uj8KdR zxkNfkP<+ujGOC0CCJo8FCpXXD(Vjh%bf=Jbtrr_u|NTq*vF^#tjK^j{jbDtgI0$cV zJ4bBo2hDsh*r`Tk9R7c;$}Tir^^40RzIlKcb2$R9cbLW8Xbb1*aJ_Y-;>ST#D*|`l zR9+s6zv79Wu5>?B*5i;Cvt9K2KW4HEi=IGLj2EbYCwhyf|%0gLQ$f%P*uS zv>&u)W{<0{7`5&EwWx3b!-|RF`T5Nd2cNpBMsp58V*2f6!TRPPrSJRNITo;8ViFe! zFfKjKts;sxw4W0)y7i8$zUB#bt;_P?)>SEFQR?@^L58%$H$k}Lnje!V`zP<$wqK&^9Y`#B zcFu79ep0TW9aY>pSYV}GO3@+lDrd@Q%jKk!Ti>{@5s!Iv6cHdvwOy*H=wZ4!vY+fM zZa3`U@>-qEn37l!b(QVf^sB^TfE8~xab>o6k7ebE{dc+R(p3q12Ib~(uPZG1Qj~L_ zVcFgpS$_1>=PB zKfKdN|GMVomvAq_dxPYn<{F>}8O6r7W4P>YcesziGir45Ow*M5`%?nOPq0U9j!_-l zY8Hob7VkBC5HSG@J1doYgUk(D#LN5bt}_|ppuMkig9 z9BO$kNp*micpX1+GiGuqJZ^G)bV=buwrlUkb;+;P?=``nzRm*`Pubn@)}m`B`U{CX zQsT}Xe#4d3D&Bg_YBEuZ=A4=aRi|}^@IL1mztw$eg1~!}+JT=}`mpv9|F3LP^XSX{ z;<*mr@3fU`&v4>CCcL99sJYZ|!?MDmed)knO&(Mg`3ctLq^PL>P;iJ&(9BYx{4L;; z{sQ;#*H3?a7XYa*9&ekWf5*X1okYVAoD^C{VT*r>S}jF%CT*O!_R^Y5Gkvdgc@OW~ zwCo@2&2xS<#AZ%_$LQB)vu6ZNpcqitO`N|@Ik7;^R6$%~pG+{Wy;C`KNGYY0u~&v! z;<1Oxe6H|z!Bm6gGx$=-qIz~CE+Odo@ARybQQy7S7>Gq}*ZH5cVk`y8yEKQFW>88{ zgN%7Cmo^Ks7unsFRh>6g1=|o>!oi!Ltj5l#=-|m-Y-0mdmn~AR%?Eca=YQ&bt?}202#Yu(*hum-(EHB zD@&88S_@jlzr-)+7d*#MGq_AfKCOK2F z3#yn6fR4Fjx7$q;{U340dnroH51)> z{ah@oa-g3sfKp?amKU2Z|Az@zLNNFG&Jfl6A?chd1x)jnZ- ze7t6>dfn^t$SW!`5?mT8JUv}*kuZxQpTVo4^G~7P?M;pdvCJaixikZFRE$5C-VVbm zoU+sbM+N=i`&W|=hz|0J|rLJ^5`!%@mG{Ya< z!*i$!vhJ#T9%S$~uAbFaV{Z6;jtm~yNNIrtYXRw9VjVm7LUXXWNj5ZUF}@S?u-ear zw;3Xw5)zJ;Mf0Y31kH%j8JltBT+F-VSbpBF)WSgwDmtHC#T*CC94 zW5!~%vnrvRnE~3?&@*l^NvhbbW7`1Vo>JRQUHdi@uFJC(EV#(C)T+O56*}kicJk`Vl-Qi39oCnQO#yrhOlPdU&#Tj)XV#xz_sS1B`&jgoW`uhllYjQl zojg+#Ef`YS)Huj|QYgFE(B-qPN=(;+*~?c|MH?O8sD((=D^v3&nf%-?`b4lNrI%xn z>`M#d?iQMpGgomFf9P!vZjH!zJUeTWQDI*m7is~4vrzu6v09gsSpfh#CuN|%XThZ* zvVzXkj2U0f*RpC-LgQfmUop|C|Es~;)$EL86D#A`=x5B)R{LI}xIiko#_pOpOu7OZ zDS<>+8ao^aeDqCq<&@ePy$fTmG_q><%x&Smu2831S^rUX*hR^_=BKHJv4$1Sw+`LhT}gbh<8Yso5)%^?hxsAmW+mh13OI z2lvPtGK*>Oa^FRm@I)Cwq~V5W*R#Y_M)Mf4-cp{EW#jzgcp{$TKv+5D-b^XQ#YiUm z5;pmG=bnx-v&u@}Nlc&Aij)#n$3M>5_<6BpR;G5No#uE<*ix{w)cga&&3|XkK)AlF zy+5K0-WTgrBc(t%$r>pY8+;34sL5!r?MaqGP;C9Qt3??EWK7+myTyzG4Q?+c?p?uqF_GPzC** zekQ8e{cbgLdIv4KvXtN$be&PY*{(d7>6_wT%w}}&B0fe{;P{yj`wMo{`@trN3quwf+k@kLcSxZVq$ZdGqU}>Mpx4j zxe5Kz&#x4?HJ*A`lcu%C^t&bf-^TJ97uW@y%>{!} z+MJv_ER9e2C*VhS?W|SxUU%hD`Z{EU2j-Q;n6r@KxXwW{v$7r__K}OMUZmvw=dnP| zcu@&>6}{pg$}ZI^>MRY?nEopNWX(tEOr_HcZ#k@D$7M5-i|Cg}u`HH@bmt1F zJT$@MkTPZTz&S)q8SSDD2yW%6WLx;?;=^C0FQWO7d-3zSJUlmMs$W6LFh@<~&N5GS zgjPy)4O*SOAQfcYA*^&&H0GmTmANMrNF1#`gI-njGvQROV{qHFfC2%`(Vv6!z`PEG*&Sh4=GU7bbD zD%}Bc?82e5efgnGAaA+L^(qTnd$VzY@?c0CxqdH$0 zCRyd6TbLE}E&Aa-<1r6vkz+YmWTDDP`7~Q;jXp^Ak&AYl?7fC#Xq-X(7iJxza-FbscuB%3fZ|P7`Z!4Q~OeA5oT#>oHz*a%JPf)n4^5HS~o80rdGzLB~nf*(_pc~3-|2IX)kTK@MpLZH5Z*}7IfSFL0RZB z@K1y76M>NEWar5|8*!IMXr}QV>+V}9F3KUQ;Hsa2D7a`_%C}-?UyCj5IjV#vWSBDv zQs7H}_h*!6E+_f;!5Lw-xglz=A9?r-+;O=f?hv9gU@Z8YMV%#uzM&IT<5Y}Or<#`I z!rQ^hs)6-LQd z>G`Go=|w#J_QAo1M%liGIiZUtCwsl!B9?KO{5hel|K2)j)%5e3CnO}#1Ow~4oyrRM zUbXBk=(kUd7CKOXDP>!9D}{S)&f%fugcH+2=wk!!MQ=uA?T*LE%s8n(1XVBhyTFMowpA*RN?Ccc`5PuB?$yem9}Gr0Ju(>?o27P*1ykAwcPuDs-yZ zCPNz3vpYUrW%Tel;ROz#u(-!UPyvm~Si(AN%u)#x>^(0}TfZ*Fbwj?vVgJkTT0#dr zm>{Dp*sXG2NX!f;dwx?Y3v$>G^;|CO8k#}N^JodRN@x5~pDXXC;ju66SfYQM@z}!# zyJ)osV#VLuNP#r(Tr_E;$#S!2Y*3**(`*xA?tyAig!MTG3wl>Uy-&$xqWLMF7#91~ zn?KMJwal<$y75QDC8;SI?CJrUjv5ta98ags(8b;e25^UFsLfaRg*6$dn05Obj>zW^ z0-Vk~yaUn4D2nW`)+um=EmOWZ|q5qi(Lr{cN0 z+JoLdMlR(bx;`kTxQlHGA*N3*}*=+wPa5n^M(KZ1M($XKE-a zGud@*w~eX+4vZ*_cNu_&g}Iyo>-!?=X)R!?`>e@M@;J@4DO@h4Z@lVZnzO7*wmBYP zY|`?KpIR$Ch22MaI+(Lv3TIz0S6Wk2vA+;qnk-$1q5FS)oH)Z1bv0rFXh<@-@aCr{^Y;{!3dOuihD0f|2)6rWE_3NlF82M znW4Jo7Hic*lS(=w)0A7=i!&wW!vlyeCBO#EMhfTBQR#e4?n=TqT}gr`h3=N-6uq>~ zPR$FbX|9S*5N0u@XIQDRXII^VdTAj0^{LP=S>Sw37*(5nu;E?*Bdz@0Lld93RR$GN zR^5YD0zB3Hz4venx_Sr9!QD&a#fDfObIm*g?h`D@DClHqhQ6-Fw*XuWX~WE35^$UX z+pbI+p=01JSPPESHgs4nI0#v_sxu?{@%W5P3~)Ql&y;gHr8WFBTi4tIFYaUYdItkl zqvUfHIG>Fn?ENj5!Q@7td)<4~kheHpLJ}^2jy?M_doV#Ynh|tf#IhRWIA3|PBD6l} zeUcJ4TJ*pQ}MLl)4tKE)4IvxvH0Hz->Hl((Tg3O3CcMK;- zMS)U@mD!kt`3Asu6&{SyQnbFd0iB_oNeyC{5NQ=;$0wDqDbC|!oW9P6l!Y%doaUeHB^NjqB zk20#8a8d+?ySgCI_76%TvJ|@DF5Us!JS!Z9uMIVnb^{bzKV zx?Lcwr%na#5QcH~kdL*&PT1sg%u)l+_0CznyI~q-EyoKA=7zj3norp8?e~x&w#H_M z8djFtlKH+``b`-^fyetd_ZrDBmd-yU`@0Hs73I1Zlfy32*TBDk(zBI?T=ta=prVBB8U6 zse(aQcxgcsa>nUfR`Ja+BUqm=hx_pPB)yHEMK(m35+k@yPM|#seCj(IC36?lz02gN^Q0whHkNkLzRDPJ-UleQejw8Br^K=`dTkDngsl zab@CZ%h1i%oF{wN+oV{eK!WCjgGLP+txLEXr%+4t@*IcFPh7ki25ldBX|pWO?b{FZ-(0KnpJ zfy*Una^a_gaug7GWG6Iqw8%BYwbz_=Q@5h_0plUK_I7S!h9L%Yko{q1R(Ukr^fxxw zO>znuJD8sEm!%dOy-pNB#+_ePUC#j(^map?(RO^ylXy1{*4SWJaY`n_(CN;(()CHD;ZkIt$@JOeE4)@qD>AYgUWw+Zi<&{csiAV($3`P@(4X0| zOO+oT5eLXf;2;jKDq*cy?zH-7({_C#x1DrO)^jNfQX5m;g;Z|464F7bxINJoA9eXf z8p&(LtXy-l5<|hdQDbM7Y|~aW+3J_CxIJcJZxonq#JYZ$#RgW3YM4QX39~EAxqXH+ zwJ|cTIa`@|_^FJ)chb4b`NE@Z;Ztd*S}R}K4_W#8GNvGbfW7OgSR+pWSIk#mJALh< zJuaM8KISYI?0FjIO%D~|GfFw}J%N+Rl$|oi$eA`?Xxp6$B}$h*UWvS?ym`q4noxxD zYRO02RA2YK1-H?}%E3u@Z};ppD8lS`%&4#&$Xy9WWu&1*D#gmse|q65CheE8JVr9 z@nXk)f3o8YGHzePCli`)Wk{)09aX?2I=QHU@v`*lQxL+LADQN z-GGw-Rcz0Ai*5WWf|;M?rk4(292OOZ@};Aa7HVFgT_o*YZ}@gz9FB{UEaxYFT*jan z8%??5E+x~lLt+z=hPBGezRZKwhGoyG4}Z#Pw0V$8F2WZXPwziw8jg#)L22o#ReO36HH^mWCrt3fN)i}0Ea&pUyG)Zo{oUQ8u zEA-qD8I)OEk3+C)=&m`y3NYmosa^QDH3~IO-55HR8g4 zs1-D4Q4C6o_>z1R`Xj7ON64tLsUq}j-qkc!)HwnExI|C*i!ov6KqQN@?%lP`ksJYM zzbm#sKVrpE9GJFT1%``H)+Hl&)%NhPM#a{Bam1lRh^4yJRj+z%|JwwhcRy3n@q6wg zR(qS87-mXFHld-w{LAQPa7Xuvm?FGmg?sc)<4t@%b0fKr_;K67|7$|CHK($B#p_SI zt1u3Jd04;;VxXkZVi<0@=%sp0l9Ehfbh2 zfde%=qM%gP@L4zquOS2j3e-r+$nl_Dbi!X8%_zp*!rnyL%~T$Tsk!%feEP zGCqP7bMvk<@TU!CZ$7|q6O=a^0=x-Ip+odk{6h}e>Il?deEai#Hbe!6k!|S#9BW%s za2QizI?+LPX~oRSQ{!&ha!1>vprPD>yp9L-{yUA>zd=)>akOKoG;a+KRP5Gde74@Q zzt%)sDOyFrLvQp4e@sVQt|qJ(H9<&kxPf&tB*~9Y}!H+(Wu;a z;%@rZO5I8Jb_o@DW}FW>`yx7)aQ&&^gS@lKN{urxcQ{1`lknZY6d!@KlO_yN%-5eqpM$-Q9B7c=b;^I%7hs@-Wj|9s(yNDJ;jW z+MB8@`h9JU^9{hOy70vaT$9t{=Us10oa}e}rU`}v!slWDG8wl{HpPiibx+nG-v^6& z0cWl(0Um>|F}N0nS~VnLjL40Nwg92V9(iVHF-wLla^;HMe^nTD0VA?~d{A61WeiL; z6lC?Ll}=+V``Yqyvl`EaHPn;cH(neq5_^FW8aBmScjUTcNJpyd;pTC4XNP(v6TRB& zZVv>0q){4(*%FEYBeJ7gFVNY%P~$_*R+XFjTN4_a@x0Ac4%Zd=5uR3VWjl6Py8f#x z7^MEJz&-L2tc(@}YC-}A7;C~B4%zA{C48>I@-YG1Q_G49<$MCRH}=Ee@~!14#4$rR z_mWRzhZBMooqIC6xm5VSK0nis&N7&2&}OR1?ez84@0+!!b(_N;^8-RhS8D(3FM4$4 z41;yh(n2$lVB4tAAB^;15x6WZw1}&aEFevP2Y(`*>?fZ(ESaKjg?lMlzGkjq`-ZS@ zMmSK>3ARw9Ok%dL9S08fpW^LO)y5u?{oYge&t&9`veM(b9VH9suExyXaCi1j-8;Bn;SAQw*iHHI+!o-0Ksv4#Qwq8I9oRy3{>eZwl@JI+g=MvhR|Exb1Z46){_t49&h@9n-WaUvnTKwt zTYpk`V$OL>h|Dr@1%$^)Hin^5RhHvckF~9OOt|<6R-2setoxD_rotZ49(?huGNy)| zxZZ`~>+F38SH^*#h9s015Kd;+$p84uum@I6FDc3pc*n!!c&;=XRgl;pNY}rG14vu( zS?`&vP4_ppB#!y*``%+thffg=c5_1?Wx(rF1^vs0-SfzKM; z4uVkP1y|Ev(xQ3@U#M@idQaghi*tf2`PMlJwl)t28@G?>m@`{FoeCbj}NW$yaj?)85|x>?cXXU9$L- z=kW4!ZiG@h96tMN@`k|$N6$m^DKHf{@Q8CSMAt|{?%+^Md+Ym>xciX(^wUe4i%>MB zBn(}YJ|^#glZu&OLeNxR9^jh=jQ$2;KSNJ9^_!_`lbl9bVw0S126-4HVG@>x{rrIR zD;o0$4(uf;!nZdlB##9i;a-LJ8r0Hdp-WyDSmI-w&$YE@;4ad$AHANf7M0Q{$J}um z3|ahEqBkApJ`2*}5u-2SIi4`iZX-^}R>F7VJf5SqHF6$QIim@96U7+NwXFsjR*AvNP;QBNx?^$in#zQ#Lpv|}LhtXd2H z?b|n|!dk?)>$vXV%z%q}rgpyW5dQNbU#D4qk$8+bu1|7H?IwNPC4Io45>4$aVE*Mv z^GC)$9*&lDAQ?_uL)KqY`2DQ67^X7buRO%Uz^zt6R|rsgAqq+Wsz8LB0vCz*v^tX> zW@?)NCN(GXAt?REjzbuT_^)L-pVP9F%foIq{9dp^fJ%n?nl5x$nxCK~2BBfRk-eK> z?E$Rm+RS81_7w&uQ^m7qqDTPV70m;X%mG6U7ASn{as%1F#rv6Ebpvfs4 z@G-vp#GTboEbUVEx)Is$Gyj34jC;6X6pJ_X~^sPh8<~ z+PN17eWt54_PUl3<4N2nv3Kv~b@bAw90Bs<`Lw2d18j>ae_>$wgbXpa7?-`@6U4i^ zFtwXXUg1$2rK^bkq+nP#oqzjQU5SyL<1}3ZrF~JDIx%01FNx4ooVYNqC_+*OQMi1r zqpgUutPUz;c;-W>Nl-l-7(WHJ%UU_e2ijjd@BK_vt3?&J|Gw?ah}me?+*HJmoX=s$ zH8iq~y=e6i*416tX?|3}-T4Gd_ocv`-oJIUbsW5KeO;xm;yO>gln%3X%ni(_*I>s( zL(y!YynMCu(Z;!0ww0U<4v%^5tj{|lhn!uP#E+o1tp=4A!r$Y3J@)0aW-Y(BSBlm+ z<)1{Ey-|9?UJnSsxQJA5aeRzagJ6}_ zWd%8R$h+_RA&LG;J9EM~k~mK&id|@qEM@g9!7rY!>_;_O6#{db&AQ6HG@S}SFx%O@ zY{cS`#m40`mtb)MHpce%w`b1BIRMjDfrqe7N`&A#tT!1E*Pow{`PAHCXb(EOzOI_# zM@2XD3SL5Ph)g6Uovh{J+Gn>BqBDb@>;n;*a*Gx`71$l7kK4^MPWrq$mm^L-f z0-wBD?K|bJJdjLZLHGn}F8-bY}?0Zy1!8oKFtJKCr^ZKVVYC0e8bWU&3P@79E6okobPq7HBpT#>prk{OOr=Ut# z*(@6#VU6mBcto0mpkhX5RGCK>DKPEgDYY|M` zyiTZd2McO&?`4lIgaVZ~UChw6pDuuo1*&eC#zTU)8aMQYC1+Awhrt3f)dcUCbvG8r zEa173Hka62ENY#++`uIJ#drT8L+}~4a{g4~U#r&@I9+6QfW76gpC9EkIM+a{vNss4 z9urOJGkkG6UKk%zaX}e9>3HaDZLa*xOsx;30o$hjqtTRJi6zSy{N@tMp4?6u4J9Fo zMH-KYOIE*C69EDT`{*c=lQ=4?;b$|&n2YTx8koh46Z{VBcYvDRiinF6j0clH*U4V$ zOVrJ3gM)-GzW^ex@_sQ#b$6L-<-JL{Y6%F(e^8b{7Q}@;!hgmy3CxL<-BWAecyjh& zeNqutKlgAY3CalZuXIAIrkEG3+%y_X;50PDw8|$Ra691O6;-88J#82wN};(lyi4vB zMh*m-z9;5L;L(hDUg?=X4^#snPx;{%KBexj}4=L|lCUmXQ#Ui`{uj}SF=yDK6OZzll$p`g*8vXHc_sGv6s`r{)# zuGoAbdL7)2P~@5kxponPJ9n#sus|B?{o?bQiF}L+4Y(p4_iJqoEyzK*uw|hAywo!b z&S+{48{h_mPaOO@lhxjaSsqzobFX~8k!wxh~ zZeHO;W0EsnX#JNdE7Xb4p@Q{qU&GBc@gF>ND7?(DEZ+MnB=A56HYc3#Qm4(q=`d?R zPLuh?;@j(rr?0^S#FPZVDLh4xln{DQ_FRV_bFkHu#uS7Sn1N(@7MDFeV}dkmr`4%N z5-E28Xx!)5WN(j&4>JJ~e|ItnS+4$bs(5v(7kS*&;biDS9zB*P{%Kx^BkztU#COgk zuL}-;RU0-O55$T&m~Onajfw!GYN_pZ-<$;{-0RtD<=MzB2e^VFgL~camN8$O6FV1> z)T4jb$K^9el6Ipyn9fO|^N2J)yi5R|6xf~tnR4?Qw-kPgQ>{7d{E~M7j~d}T(!zkH zCjOSx(}!a#2%s<#wYEqT%X3eg^vd&w{Bly7J#NKMmFXv~tMLhJa%VTb!y&>iM}%!Z z&Sj%Tw4#3BdB$0#Bx)z=f0p$v;mP6n@5Tz|E|i@jX2#;L=c%eLhv=_@UMm}=W~ni+ zek50KIsj=yl4vtroZJ<3bqb&PfXMV+yn<`|mun@tw+D7Z7;nvmpE;3OvMNX{ZNaGn z{%Ym3vanGyZwd)vyPzX|U^lBV;sb@>-O}7N#N*Ukf7$jP4frA9_ZrI!%d*~Z3<8MVs>&Y zjpG9m^m3ghx(}Qg6lri-0KO2RBLB1h=kxFK!VEWl+;$;$44<3W zoW8*fYsFLqP|y*e*{3R=pF&=%SL|LzUSWJj^7$wapIfqJU-`xQXmX1gMCW05Z!uT| zKC0jt0}ASvrJ4waLc#cT{xPX-{r-p)GYT>?S`nabTTd$xpalwyqwu;z52~2|q)lQJ z3i!~BAK3K!ZM(vsD%+#dxJ#x(5+mIljoVIp_ znG{Dcfzpk~VI!)t+9d`ZfbP&|Cs8ZZ{5332j{aw`&xj4ii62KPCEoO9Wt{(5nON@dIAIPKD>;p1nz~aTPya)<&gfnL z!SWDxzc)KMt1v&;#20`SZ-Q$EWZ_yfP%fPT$%Q>ipcBYmYJ>i!uKWetd72C|N8CTb zZY`qb!VEyPZ}&aF7J)mN$}mN6)9pspGR zEh`{GN6M*|GNs?@#}AX&+~bCm`3xgOAx+8ZIi`k*;H(_mh8>IX?UL>mf+&mnDMgyMxzaqzdjVXqv!Fquh## zF9co-A}OxF5U;gtvAr;1OCSJD?F82%uxsv9!PbgwEpX$C;{lHQ!`Q~6D(j_6!nX!H zM6rJz6%VLMnq(>zBt9og08~b_Pf?PluBO?LdBEgdL(n7*eCUAiCc8VT^12BGsG;H_ zQn_I|AN4QY9O6L$k*+wleVa+uByPAT+ev7QzW*eZFA3|qJlw=!n?C!et`n``_Zflg zXYq-kKU98V@n`*ZY#%D%6iO9?m}yGad$U>=Yz}-G z*;vadme{Qyk#X7kds^vssa$?ZBLfE_YfhgtUdqgh_}|oRg63hoWgnf5 zwA^IMd9ACq~odWhGfI4UN|hHc2VU|_vh}99y|)L6Y?(taMu-y3gF#m zOo4k58dVYY5&H4_=`FtgqoCr)(C@ZbgH3T2pCA*R#b9OUfkB!1*;VGxmG0HSWjbKFsUBAm}2y(`%cFc{?mvqs}$dl&pt} z6)Rb{+~Yu%v-vVsKWK5`m|IagSL3kt%{}p25Rm8pY8kj-@Wp&rBUTYQ>p6?x+8E!4 zBCw>eO2k{^Ft2)Wr6^k9puM-+3ZDSrvaX_^?3 z7_^h20^2KKw)-tHgiR(1sROW6psh~DX-N+6OAa#yVu@?+%oV+lK^6=M{VlP zr!@;CJR3P;WIQ#o0*=b8`OhJU;#?Sp4~cNl+>6Q4wj@^~U$pC91XPO=B(1z^%8XKS zG|WpUltLTr35l)sW5pEGNeQk1Y>T{vTOX9b8HxJVf4}FwetnH&pnnTHc9=C|UMxSb z5T!n7YH)RAJR(+xn?v)d;b@Kg`F`)mqqf{S^Rgkb)4NKJO_pPO-zab;$hC;B0Eu?H z+3I@i9FkUD1!qsU$k!=?_Sp=S39s)(V{wRhI|`3Ri=9cb z)9B1f|M`~Ox&U1yNIw|nmUljk!B%uU_y>F8R^{t~44IEr+YT;HZXlH`eC>{Ol7he( zG~o;Qeud~w*rKtEk^*y&cWqhe^frq21!&E|ffbJN`Sgt?wmzOn(y9hKND@_m zSnBR)B)90>grz-cuS<4%_RLLa)e`Ly1a#$-Xa+P&vW=*UfX%YBv5tx4#!|lHw66ly zwZSOWfc$Ar7w|I!kt?${`o+3W5z|Nl79FDxvHeJsr{ zaC@bt*bZ@MLvJP?xc{1rjE zq-Q!Boj0g&rc?r7XMxe7q6zdyLBTG|0w1=`Ue`Ms5Y+(1YfVrW;h@|V&nYE^80 z-m>9R90~+<9;2do38UAhZz`{2m7&%`KXlf#&|R@e}aAb zNgDTh6KXRY*-@j4=|glq>UEGypRZF?!#yH3KTNGnIaNed$qi}UF>ci=Z7$_5v z`9|$-`bZWOKK-sR2JG_Jnu7rUw1!n&=o0E}H=Dm*BE#(UgJ+c%X^WvXEc+ZT+0+?( zLek2VP4Q^<#&fVTfcJQi-4Cupa9+(<4t)t{JIery0EnB>{^dRuEWt9+6}U;DLjwNQ zn7GI0C>u!-Fs5%oX3+HTk_60PYTKot!+>?8=6ra1L_H=onTUY6!sB}hE@#q06D1q# z4^J^kQ5Q`y5eYzI$nKn^W#;LeJcp!=zWj@Cjs{h0AO6;Sr4LXt}L9MahzUuCQto^#Dzz=dt>RLb@eZTSmf?VH(^0S`pVP5NlpzvF1ddu_A5 z0CDddnjbh~^Njp;@sYN=z$sA%l3noi#NokkC6v%&Wl*l!R^&AW?DP$j22hbYB1_iN zPRs)gs-=}(8pVEpK$4Rc<#uq%wDL;kMe?@VcT34Z8rctm)gQK9>N>1Qh&KdN-;q_A zTDqHTJy$c;c%`V|z_1HLs%KySy*bb;N4oP$0UikQQc%c5Xu{UG9Q#&Y^7#{;17r?L z9YCcJi;5?MJCvo}J186KXz9PFbjt?_<@oWgfat=;&+G4#Mt`Q|%tZQ`*4T&S5$wyV zh=Qcn3k%;@p3-jE>MlPzpQ{>8ffN1ckF2_8jDCRnb|4wngaBAK=XkrD$IHQL17TrR z{+NezbK{MgAN(WF1NC0@kC}GR)RA)z`bj>h7`f#tQIgB+zowFVK+e z?Oh>%8*(^~rEbX{8+*G~RgH(s;Iv2ShZ8E=5A7PX?uuNMyANq6Uran1$mZ6m=~1OpXpmB_pbEK5tr*?D z;DN-4?u)I5gnqQ(LEEu_1f}TJ+d=glyHD~S{Oc*idzURDrR5m=4%`lH-{X`%KZX0# zQ>!PG!p+P zclv!K!}7Y<(&e7!%jTxe;3>3@P!1)9A_$gmjTq z12yBrTP$9!2@1I!``Z_$AqATRF1M}Fh#;iGZJo9G8&5WZ(u5&Cc+H5XK4r7ek;nGg z!Kn#^i?a%p2nsw82xmK>!B{o&gmrG^LJMH!nk!@X>F$V#L!c<|oH^W4)z#yJ|D zC&Cs=`*sHNMxn%ZwJw2u6bh`5q1uZrAEHf)be4Muy3j>5aBh6Wg!JFy^heTaM(CmWM%sRIolqvE5m8+;c^-+j;)7j+M|Mv!`tGx5 z{1Cc!jNBp?-)cKL4n(4+cG6q0ekHpyu>Sj8a~02YJH<$aYm=ZFk@O4Fi)E_p{>Y@Q z*v*I^!S|e1gaRNC@N4>zxDwd?7{!vO&!%-4nF1po_etELZKcqITj{&o+$kJ~33c+C zX&agilbg9%o0ss@-=oX9725pGyY{y)X5cwbCRO3=L`c;-#J4O>mbxsrX(xEB#QNGU=^z;=kfuCJZPtE>34y2$UJ z$^15`EuwQ8L4o{60;l`K0zMH-WJ=pv8I5sx&{6NL^UbMO%?R}t zx7Tt)!baA3x|KGghIw91}7}(ZQ9$B z+=dVI;0(sa4WKj9r&m?}sfkn`X8hQethO8JKt?KNx*X z_i+%(=UaadrsQIdtS+F-d1R@RjvyZDIVOzd)BZpL*Gjv|z+T|o)miwIw zs-wvt+zMiwK;mqFbZ7QuX2MOmb z2w_P{mgEizfBefeEg2)}Od7PS+N0YVkG>xm->-t9*}mv|`Qp~u<8=iK_Kl+xB{6ib z)`36MV-OxEcUl8j!tWJPuC{nQs%@)p~EBshNqC&=b|!{bj`a#O^`FfE8xU>1~g1sGN{^4k!G zzxGH1?j~W4*CeF=Jk<`0EY!_4@%ts|Ps29$`ceN%0S$bo4`SM9u7s(hJEl8=8)i}8oPitQ6UIK=0{wS8@{MiYWw6ozFH_pq%{ z9`baht6DQk%Pnu^1GKd$(vM`--Izx0&Nqwi(HHrvV*<-ew2J{Ium!-wS?&lXI-mwZauZ#|!X|4D5?;cK( z{S|A;R1hC9WHG50JCoo0=?9LA>G7vponu(89bR-v+v)U>Fk(^)`25*c( zQdUQ3qrZh+FoI%(z~#=3uQ5RRdlMj-ocT^bjCM15*qRSq7(Bn>LG9A7&q6Zz*K#`RwhA^gA$mNcq@^G3Vbfw3h^cAU zH%2(0vVw>E@n?BFxTWV@{3{C!+2-9%UJQheZ z!v_nQ{h{`jBv#OG%WEL3gk2YbLMqG)AJi8A%gYtRR3V(mkSQc9wA;mo@b2|O7iwRt za>U0$dvrPpWZD7tSi+u($qm#RYCU*pOMoy2{v&XL>0Y7exnuDzF9Jlz6_J`BLV4$X zwcZInO_}G~YK>7kzO~+W9-zxo(Rh_Z$j9mLsiP3{fzgCa{f{dT46_H;@INPN$%$Mw zb8jC%TbwM?)AGqwkTlLw%*Rgj<<_@#GcbE_1yNKUBU@xAiitx@e{m#{L=ilMz+gUY z!^8$~!c~ll38dFsuG|}asWCma(Oh$dRvClaJ{+XN4%HcSwJ!~3&iJLiZxVb0XNd`1 z^{rwUq!|Zz)SLfOFqWp17rNX-Ap=`hViml+)8qeF`P%FPE zhRp#hdCv!Q^LK#&M+au{{%4`fu`dPMY3HlgmWJL|@sUJ=JX5=fAT}EtxLrCFkGUkL zgb*X(2e68jcRvtZG@zirizK>#oAbp(vzYASt(r!#+rAo|t7otXgIh1b}`uJ`sb7vk)i4NDb84!o~5R-i&y2{kG zQr(s4m?|5j`*1zFXHd=Y2r$dK?upfpbBMNRoNAjqpuY-?koFDA1Nr=Z_thsfbG>73 zYs7}}s6KBmx!If~B;Z1BHpgd>=8oPpf?En4g57#M@o_n9tFW*nrKooUg29_j^5d+u&XCkK|v7M#Wz^xGSBa?4p(TXUkP{S@`aKwgls? zK*4>=^9eKzxi%2ox2K>|NH+!|RR*iM6dRUWHDe7W1aKKOiF;H-=}90a(fY@l7m0$g! z-FH)j;a_pS#e{DOzB4iYaMi|jCueGLJ7>SJeiJK+_-tWt=Xwq;H38ezyXY%H z@{R_QN>`-24r~fK0Nex4j%in9ZWICW zc%x)EBxkR0a2G5kei^OFw-g3U95_0+N;f_w6GN$AD7%62l>+Bb#zpm&8;)+f z)1uTQM}Kw@<$tw9>ZqYiIx`lQ&H4(lA5EV=z9}i5qZ88eP<@8Fdsuo z$=@l4#oqd#HA&D~Rn!rcwBW(?P~Aonvwp!lvk8YL=CE+TIwaLl&?03 zUb6B0ZjI7LiHWV&S+J{>FU`w--Oe+Fh$MSVvAp-^fPhy}{KqT2or0QLRao9S{FXMZ z=ji@I*3Abj1wi>RkfkhGx8{8Eyuac58uDGINwv%s=ZQ;a8)to-22~+XWG@5Ut>B-- zee;56jaLb6?g}ktA1j~YUW7j8O6ZABNKJfh`rR6!GH!GL>l#bPzK)>fvG_dW)>XV1 z&v^k34&yD&jta*WH{H+IC!Oqr3~70r*u-BD=ktEKAHHFawH=p=*ya!_;)b0bL?M4A zIcqqJ+W2%O(R)>%u^$6{$otb)5hEAl6S&tGYT0`oZQ2CjO0I|EUp8}F_m*(_J2@fY zB}jVdjx1|v$^igh3?matxkQ*qN|U1;0Y?Cg%yp<82%L50_>Rt>5Tq!P+prFMc+;r{ z@6Cev0^D}kPK{5Hm+Ii?-kA`e856TmRj!z|@u}ir@TDn^tt8h50pHWcdY2a{uT+0D zkebzNMt-Tr-n2cfcX(5lwwqF*oHiJowC7{gZXF#yoz<}_JoweT z>Nv>er1g+7od7HRiuSXa#YMov5>6E?oROk5TRE9N&t4B*1>XwI6hdn7FXYU$3KV84 zpdVnCk#h*!gJOLX5it+i6!P}^@~bHNdeu(`C4cdHHjk({)qT7*pTS$8X*eJon!fHb zp*%wClKCja)YE^Rzddduj&{(}n^;F+rj5p>$9#QkTmAC1b^U#tHQH&t4_XXk2aRLt z7d2sb9ENmBU+g6;5RRezbE?tT(Z|OE?d}lQ5!E`H=ejc^KcpRNo^r8v$KabVGr^{*hglR~M( z;rXdXaBdbxCZ0z735_12GD0}7=1=p2>T?%;q-bw3rzVoj=&-_u24ypLwVRaQX6(3) zNG^S1W;{p{*c0^NUfr6G#lU;06`vw(N@~S*mv3m3>ikptTi3AE4h}mD>6a=z(P&8O zi|>n%nw|wVQrZa5rm{&kE{7pHo{d-1ldMaIajSk5~>*n_)blQL{!({B{(P!Is3FM;Inc?Sggv#B`}K_r{vK z*3O^IZ_>X^h!}_rJanv9;Q!SF@q?ru!Uds5=n*GG18hSsJs#(U#zEh#zU`HUmb zxt0iYFT3;1k_D#vLdH#cwD{Y+Q;QG!gG)K>-JLs5H~5i=A_N=k7?|nqKZt#}5t2h_ zn(f6eAVMTKGJhRm<5)6%Wm>K-DGhDR2z(jWTc+4%qw`$pq<2_ps8QgSXbwR;!h>93 zNTo`<`kf_Ib30p_clhfEBI2awU2AiBT!{J`TN%=#)HBhmYQZd`uF7BLVau%~+3UKm z-usy7h78!GXqg3jsB2JqwE zzwu-98vyJS?`2Ul4f0fnq$D@X8aWhye*EMP<0b&E$-MS}%X)wBjC-jiIWPUvEp=9I zz+>?_E*W8ZWX}${S5r+rcQc?9l69N|8J)F^&x5}$=99a~WCO2VpZgg`AS zqOVSdoMT8`)1T>{cBNRBFm72}ikRmsVmWlB72(gRw4h=o=ezl*>r?SPk>C%&fyXsu z4G|e90duZ9Y!=a&g;ELAc2;3IF3P+uR?AnZi;%L4L)+Z#!SKTW?p#)|ppEeu;xp^p zm)97jfIl}CAYN?dHl*2yi8>`1A%spC= zePs8doCN!f^<_ua`35vcHmPJ7A+pf;p6K4(sQL#|V)mg4{)yRpuP(GIo~Nz~4$b)6 zv;t}YIIrV(#}AV)rRqNVzA_NDC09@2s(ND5UMRKEx{-M&vG>qe zD>V#f=`DaDu*rd?i(I~xphj2vQnwbh!8F77UtJN^{|FXn} z)`4P~JGq2)(tprBi9ZV9Ps6fAK}2#Ul+-0gitqGIS@uL}YpH(zda2lz>paPWGt>+DYC$b8vp2_3rbj|bF zZ0fVrXQ9VRrY4+ZU)@#>&=^jut9s-qWLsvUxPVPar+(!aP)x@h#k*c)^@%}{`$%6w zk=?76sx$7bqy7;56X#*ro2vue=14J6PN8Rg3w07Ir0e!7ZP#ywoTu0chbe}4VG=~WJ`3)96p+QWT=dqa zq7~-z;MbRNwswHTMR;D5rJI@3ZSVeHh;bH5WCL)o^0M?jK{m$bNsSwK`Z1lSZ^coZ zob2~g$_+g0i>AtIADfQ1UAP%_`}@+=Iwtuh!D{V6M8%LbQ!TF`u9k^IC#%t~@ zCO+nkha*x~dq=MybmOPHeh^qU`w_?7mW-bd=&XVN>_<4zJd!~Lx{qYilN-@yEt&v{ zbU9SM(D7%6uXdfWw3n9`O=OK+7xko`QsEPlL~fc~WlWl~tuv?bMVdJ6aEoJSyTbgV@MQU{ij|`)ncjnco9Y zMk)Qa7}f7uXHudaUP>PxgX1@-w@LHpe87(e!m!OdH?u&mg!#)(3RE4wl0F*aB@&db z|07q9X>zOl0S*7TS5HWNUvAerGL9$P(b{(|ar&ryla!JY#b_^qXew6X*Tug z^l+*+y==I4LTV=Y>WB|EOnd$sG`n9I?wVkvYoYP!k@~tQYc7kqx6Sv`oZ*rAOam83 zML8w!WnUoxyXqrasnT%s8EFVl5R0|E<( zEClG%Aji?Sby}78av8aw+9PC0OS6=vBe2Hrt`fG3Dz6!rD+i_t1hP~MboaLMA5j43 z$u!41DGi1$vV>=I8M)GbunGC_JfJwJ1bdjc5%Z72rNbXg@bo=1rBWWba+N^Q_cS?f z91xKIL$&=ifsg6qHh<|fkd&3kc49zVA5m1+4Ew@7`L9Axg}%5PD!_Q|4we{h`;aifm0+?0q7d( z<3$OJ?Fqon=v&lZfV=~Jc&-Uc*5(Uhvo;_nAP9^Ba8v(T;@ zzqijrInQ#5ACD!TgkRox5I9h3fIxPw4gxrNqGtfHnSr+VPf56&U;x>Iyu!lL-ekr* zhz)B7nlEADGNoijSVnLo440txpyLMCu4p8DSi*UT{Bx&n_wuAm5C;>pufN}OC5%hw z^@Vc)@$nt-94oBQn1~!oi!Gd-{RB{pTV+Xohwy99TlRIS)*RXRk=Ogw6=a?N((qXO z8cM+*?fgq^{v&nRO&|chH;z&Rgt13I61m#~vD5Fpu%@F&n6Px};g;U;Uhzct$eg>% zt{IZ3;Rr|V&d$yuYgNz-gZ!17TiRP=V`Jur6_|W+);58CQ;{+i*=ABactokaEv&;|K2vd*&Ynh)6RY7(l;1@@ z;~nqh(qY682Ao@>iec&Qk${Zy`hsoqP3S4;$f0}Mt5%8S`zbwr>E+k6i9Y5gGGgKv zf;Ol(6SACwatMvdetFRUWW^bKcpXAGJr3^Z9<*ig4+F6ph&2rVVxCg$ zH>!1Cb4M>CdY zw49tr+f`px^OOZoT5!)VE;hTi^(Ku!)GYymxHrJ=Z3DReSW7U%7^QC}HvwQ=WKL6h z0{*D@n`i%Mfz?)U3A2hmA-|D@kBA4&@ru|j0&KE4pN#zUKwP7*F62a+DWrLqc z2L}IaM0x1nIyrevD|m`t`(>!f(9qCq^;?`S4h~K>d6i#7thG%5$l7^K)cQmku)65D zEM-Y_TsI0sHF!-S{|w7*15X=gQUb4B)&7YfZ+mTff|oTK&Yg7Xj_3A~M8oD4B&VHc z60UD4IOo2AB5vx(k2kJw#;*5qU2rhQp?tc0T3vY8%5cPNi%uVBRcoF#S~TVo35^w6 z&*>WL$MmGF>okE48@sZb$Vm#o7_`5GCZ83M@=`uixNIuKs4|<5Z~7?iCoui*6Y(N_6lXI=xIaAdfUyRMC*PpR>5(tGetZ1M z>R6*409h%y7hy1h{CqtmS$jEP zk74{GtSn`=JSqv27iqP>?6pz7d~aW0H8cGP)ssU7J=}lz_MKU1dHIg5_1$ZQl3Q)5 z-+kDE9b}mhQV#%!l`nj4(7dJ18j2ylLXhnH1XVgCD`kM$1lFG**J`XeYbsq=`|L}k+tRdLgv-r)4yi&pG)6}Ed(M= zHB5)Z-r!9FA8h#`wm-=?fC|Y@MyrHnQnq#8Qyk_peq|)zfbwqJ$B*T&F|im5lbNEy z$VDZeflkF0=guruo;NYHUX8?*6MUH^$8RcIc`Erh!tX{()4F99DEbe_BdZTEw03Tm z1@2?oN0L&~hZmnR#p*4W`(qXNzhCX<*fty)dV78(P>~2b{gD{a^%lr`$R$fdSH3pXk(gQ<`#4>jH?6MkKI_``uVQsK6(z4M!8~G= zR6vT|gA1a7ibHf85Nq-MO&;9-`yDiE{L9As?m_v4q>}KmJ6*>mLubD=Ui?h;TCu$H zaBz5kDUCzmbxe0AJQOaLwUuA+%IQqLc-{O=$VWxQ&W@*nBe(^a!-U_ExiODV~ z!gy9tBYfZObZkuiwO=lDJMFETD$~4#$Jx-;kEJfth*f{o$40FD57)hv;z`~fzwqw!N zdIXKOebNk|z6oZ})WBUPAY}7!&zg2Y*o5x$OW1n2eTS1|1+fZ1}8->p0{! zh8yY&D+310PaH8=E?HX!AJ);;la^iYsumBuWu%3yx1}HQ2TW&Yn}oF4iddUfY&LW zKz2|PFJfk{P>IQWfX9sV-~|c3zMTYG%#%^3yfj5a2{O8axGTl(D$G1dHMRPPdNnBd z(#pP&931ry1mpmsX- zTWON+UAHRGK5%zK`~O)~P8rN$Qw)?i=nLQVe;bb;OHr+6V{9V^!dFoS)(2Rn^q(8g zw&I%3zYQ(XJi51Bf zViBj#4Ao2f$>qXFST*J8J0kDigE{H$3P+u{xZo}Z3`mbdThiG0`MrxZBdz%ZpMMPi zFZHY7nZT>U?pZbPQp`~ym4Itgn}7Jpf5R5rbVElSAOl%{fad3HDrQ^uHS?=yZT9!5 zy8ZxRe-lXtxz~OI7(f2Wfn0RrM25Y5SxC70{928P5f|kHVH7>4qh?s5cP{l}1>v|7 z;(s!uD457xm%R3gACKh z%U98xFjb9>M=Es%AEX0%JoUJZ4wxPXHoQK#l(U+Dyw~ROK^Xcq<3F z9X%{{N4*uv+SX|1V*zn|9U@M=57NhlvlOdZgN;9YZ;>I*ph4?d?SI3dZzH%G$U`tO z4b92T${#ZW3%VR;jumKO0n8uB75$4V01+K`-gVHl`tX}Vj`*GNmoDfcU;7Yx)$-DS z4mBUM2#~HUeHyXwHY;JC`JUa2PwJI559i8Z1ePNpO|y~1I+AilOWhQnlo6XRoz=)} ze)>vJ(t&mi8>#=lV9^3D!DUksLX8%Kr=%1=tEev%abJqX8vYbn=u`h7M_e4*a%%^V zIS6o%m}q>lyJHO;m1wk6K0Gk91Zn)C`?U&5WFbZV-t3FD>Lpf=&$!Ltc2e_f9q)hP%c=X<0>V1{K&#-i+fs(`)qU^&nUGCX>^A z;2Jvq2er9dqiH%HS8XsNJ^LcDE zGQB@s8oCM`?gI`3dVJH+w&!fK4yFTOZmYw?7WF>a1FxqV-7lWG#gkPV+!Pze! z)<3M_i{cK19gl{|U&PN$QEG4-oQgo}{8}b){SUrW0{miy=Jrm1An~R}YHFe$&-M6~ zu#fH?A}$kf#uq^$O|LOqpU)+LmZz>?y^fbVLJor-sDnU?n%e;WUqv7Z6)e4H$tVG( zX88;9<=z%A$NM>+6V><48_*(f36ua}@IssV6FuzZwL=DM;94MvpJX*SY9%D~O%Dz# zSBW8N7mqfk8k7ko&@2QPP#d=*OCc#a7O+pRF39}fK=6ZB`M#dw)PDEA{Fyqs?xNRM z`o(xL@P9ZIGU6N{U$#ygCH$#jA<0zlyCE-c;UM0Eh!$|I{eMu4CmqX8jga}v?Gci1 zszyJHk?q~Y`gMQ`_Vmo)k;8P^@Y2DjXsRX;V1s~=mxLDeV^<~V?&#!PT4c|L-0iuD zU9Vk_*C%b{MLG>lEe4RPzSdh@o ze{u0*_wvIWPj`z=9Y~!-nS%cOD|J#5YY~tyRWpVCeN&1K^zjDFF4oa?rB8D}?)F;; zbW&@y!n+c$r3^t@0D$fQRiNlTQ%X!>7Ze>#Mw(0N>3I<*=Jt+V

uDIw(hOgias$Cgt3HpsqCV$%#0%Tl(9yJc>j-!H?ZfMI0Hf@k;0hsAU-<(b1OnwJwQ zCvai6yJjN4$r!si4kj4%geuFX-cEwfTBD&x8M_c-v#b?T2|27Y=~IS-H;@x4)+k~X z3TUkPfA}P=HpHT8T~dD5mXop2QHTHaz?~iPAZ-PRKh<|7Uj2V`y=6dDZPx~hf=G=F zNJujvAT2O7BHdsR(jwj6sl)(Ei6Gqw(%mVIbf+NQAR$Pfd(`K7-}gJ`{NoRM?!E4n z*Sgl)5igCb&l*38JYevhD=bHOiF8ai1iDb%!zdqN@w6$Bn*wuyQTiM!xyznTa-ETX zPXuZt6Gqpw4LTye7D)5|ubT$lIPjAZ2(`K~Uvjq5=SdV)8#Y!f0}46FCPC6Cmaaw+ zN{vLK>`$i`TiQfknkas<<4|FCU&Kd)&8tiA>FPSl9MG~KjsGU3->~jjXbt>=hz$m< z93XgWBV{Czo{=_iqaTwN$he!6{w?CEEt3!+CT9a5Ap9?u&@%}Q&>q)ji~3PUPDm*h zeXIM9hbG`9IZM?}uVvFGB9Ed+#>vki%0=BW>ND5Uq&#jf%`B2gYyr<$GRwwHpNmt?G!itxlTXvR#`3Iyi1;C~Mw_wNkAcJ$Gfd&M(G=K@q3 zh0rIUPUUU$S}RCtItcZds(72Lla?ssS4_=715LyzW)5MQNu)To!T?71P+@6HKJ!! z>XTC|x6bW0i6feiaIA>0rWMW_ZGV#Mll7s389HjEl?84C8kqx-{Pml_k z@XjWox83WUhT@DlB3l7i`8PWrmRdSEt5?GKr0FXQ!i8^Cqk-3qB>N=Z%R(x)z-nga z>Iz!)N}zW0{I_-^2D}LI0_U9R&vc9>gi~hyx@6Iu)Crs$2eXzw7RpA}wfx|B06hW6 zTEGuGpXk1X@7)`qw}?Jj4g;y)0T{m{*A-~H0C&ZQ#7V_}GjS6^u6Oz`Fj#0JZfRDr zB?r+jzVYnqO6kvvm6|C4Lgd!>0%GOh*4nV&1(U-_9|v^@Xgfy?I-*bBQ10Q$MsSmr z=8O|S6v^+Zh=qkX2c@Tw{`r6Wll(Mv*1YN=%)C7NgK(9&FQ1NIYy;_+ zr|50N1oyZW;vupgXT=qzI6J)|;lErxUt1fL2M^<~cb3RYhw`1ip8$lOGVzpq;dxP5 zfa?b6&5aDg)@V#!mM1in zYf4IwP6?W!IBYYC#6*@DNAK;ZVk=;zz8Dsp!dB6d#s5&r%~!$^sQ)M6gQBQ97xJaR zS<)=H$APN@qVHdiRz9Mt#XUFQx>8DAM|1;x{aP0L2a>N>%lVl{|2;%QOIzebRk|XU z6t6AdG5_BFDIU+^_sKw~cJ|R_5BptHNumMQHY_l0+Q{&D!z!0-Ye&qcbYcM&~4cOw3&x%eqs z8y4~w&W60kbY3ckE=!@`(iXCI3@Vs?eCDmJQL-NdNLi|$TA|`fzec|2nS2f$^W6sM z@4ddg?kt73jm!PKd#(9&_a3tdrQe=-HQ~#_aj?9VW-?K@;OhtR{YiEQ;NxR zMHSrC%-vIM>IbZXsRPUK(XyK@8F_{@-QBF6tMeLk)tklepv=Wxhz}|fO3ok|YPPJ=2;M}@@Z4iYPK90_hEJxHQVEPjsAUutBraq|^m^!#u{4Hd)mp@+7==$#S3Ct8fyE&vA z!_Ymj`eVWL`gp1&4NA5TloY@)J}b7A1!MhgbiI>6;_O|q(l%YJg|275`Ya-eCS zEbZ(=_TNq~Y{%bJj*f}@Q&3n)isU!bkd;!Z@=@@qVo%32b}%-u9|b7)xKGHB9_^P<^9X%k#JuJTbV6JSCk#X2pn0GIdVOzG zHargQ##@P|qMBnLhDiUReD{XpkZ$ZgkPDG|GF9aZx|V=L0|4H2ns<@u`>lYj;D{It z*sq%3OiBTFUezQf!!ikx{cDM3*8ltqz_PCuF z(tByT(E9ZnO7iO}t$%^K`+BcZDY z{GYVIrOQX74giNtCERTX)Sq-(7j&1{>UzGtgmp$M)w3*qEv$gO;P9cIh98FCC1Z6e z`5B!{K~}N)F9pU42+kCPraq^$tUL^?0I*~MbrMMAm6^Ym<@j9>xM9u>3X`T$p&^-$ ze=9hiHconU_x+pf-&;5gaP{k7!F+j~{Qm!|d)J*|Jrc?k!M7)VMai;Wo+$!+030Xx5$!2L z4kqE+;fDsr=xH~Je^N=!nImqc1HLqMDgv-h|4_i!35WIprD}QLQ2RKYkAHO|z+hdM z)Y3+@N^NkQ33nC7L|K3)`@m?=9^e}&XzK!(5cefq zSBU)Kt+E~-hIS<$w6!jYieo06czoNtV4_|#sSsGkiJ9aPriZ@A5ERR~MSM76wFK?- z>RZ-4y7S^C)trU9irC2pO3%NQ9%$vhehw>fDcX&q?$A;!Y?Oa6ePGbKZt{wte40fz ztbiRaRkG-mNust=fPa#`UzV#(ez&l|dFM+Z<3l<|yZsXz{#C*u_nW+2Bn{LF9aRvsTNPp;u0gIIDwff){kR!v`oNpGcp7`#GjNV4&Ky zmZXmm(9H20x;y0b9GB5Hkis!zxI{_O0xDpyQNdtG^deqen~-?Az_~X`043 z9s5v#YOGlza^K`ebgL1qGN7LgYV{?tJ?sU6ThziM9?~6(k?CF`)L@I#@c0Y|Pke;P z(Gc+U%bRK+W>>-bW_tL22i>WJU$;K8RlP%=Au1f+V@epa7#NFWM3<3-X9_n`IqV7i zwDoceR&#Y?NG~V$eLA?mPFm~?R=G&A3vdbhW+PP#)v9mw3+U4YyJC)tV@;ir3gACV z;;~*l0JBE;ZPwVd&E&uBFJuPrbb>llK&j@@QVocA(_f3Bx}slHfL7HZ0xw{Og827K z-6`{+<{AE7Q!0YB_nP7VqB)b$vaWDQtGQopoO!>o%8Td)DXwbc+@P2swOFiE;b_hW znELWDv+Z#krM-qn5@-G^+oUMZqhmCnR3uAIem&eztvWf$IE8boU{V_QfzpFHgXVM4 zB`_rlHAMUZGdNJEPA)J*2QQqB%GmE19$5N}%AUd^2Pfbd+ZcQ$`a;P` zre&L;S()Smazh#yXxgXefHoj*n3v2{#8~_3*V&_N;xNhihPTE;6!Ot%lzUrQjk@qL z&^XfVXu1r!;SXp?W~}f- zFoE7eiS%c3`C^SeRXM39Nm*h)5FI7Z)1e0Xv%?!|V6liF47om~+y54DRz_#zkc7diQJ-{nZ;Vkm<^F zG5)ErmT6OCow2=p29tKZ?}?Wu)whUsjrGg$YA0F!&(?SMH;g<#klBbQ&0wLA z#w&B4cpisJ<}!ADF%g+aA^K5VkQ>6} zP^C5==MQ#vD68xf^y}xIl(PrylI4lj_v%|I~~6Sa){} zdgo&aX@H7G#Qd;ybe#}`l}~@JmS-EIMtGOg^@fB-nPeUD#H)VQr9u&dc5pvE*Gd2t zE?fm=BW=qS0pb2O|?`o=0J93 zu&2w7@m?9+;n53DvJq}&lTPU~Ei#a3fkMGDhMM3U;OE?5k|;?4`z`CHh;}M^?hdGp zt8tckcuf37$v8{sf3g(RY0ROtD}N*Mn1n{Fc|21m8|jd0`k`1sUt2)jKrrUw#GOu{m8$5Ed4;uhB3g4Pl=Fd=)?;ky`y3 z?iH&eP+R-NnB;p4klH`_5?OouM0Bovx~4HeEdOO5zcM+$cgUMN)Qh{v{fXDAiXU|w z@}CUAkofa4he8gtUV~M_xghSVjFpP~dy>M;LO!b7ub*;14Pdmx6t04y7`_BYV;LMC zKBy@2+Ii7*1{QOFjENzhzTW046VT!S_?c8}DGXKKA{(2op3O^bE zkJjozseKACvAnpimR2fUX*Wf9)LsC3Hka=uC}KlVMKs$#sm!Z2qXDZYl987e^f=B} zsI*cl0w8w>Sf1_Lwfhixx_c!sQ^@J(%FEyN{VX}SRWL#(EO2>VK0p>Y6}*!U+h`|6 z99X$5!k`(U!sh5I3_3t}`ldvJiZTC@m|g?K3`!)#?*jimj9FjVNw#aE(LQhlrh{0Q zK!|m*Pi|0lfrZ zIgvh^;ry&r%Ux5SMB|s!=Lz$*$O5akdX+*HA&{p=qD-Z3>b?)viQ4WhVNt7~$jHiR zUtDUf&k%!wwZ+ytD9Z2kF2^-1qYc^7 zFQ6WK=Za||k8LV6wMXgvz_c?l2%SO{r?N_A)%ZJG0$2=oB&K?y;5%rY&vKr8ald_@ zjEWyb);k~;H@e<*DEaiE@RODz8*Y=Dx8=P{0i5q*FtB?CFoblR@MXO#?KC2y18R~~ zRjdpJQT|0FcFl{3i3a1p-*Ks&nlzv2yFpgUyzZcvT9896T>5~qwkk)dvPA0f&Ek)- zv9U3_?i;Ip2}i--^gO3lLjbN#EsdU990-9q)>cln&n!xvp^Z#d0610hPc&ggMiVEg znh^{2hBwexp!52ttMJ90Eb=biJ^LMUuTc@T^(BV2Fkt~NnVOnl>EUEy7TI8}xD?5) zYY-ygU;u6x%3g}Q}cOzjK&iv5g-#V7Y&OmaK2#x z_NifsLIEJS{maxvTv0>H-nh%)Xm44s$sAkmQtOGdbm!^e+R){3=i@6ogVy|T4W!2n z3gt0$QzjW6^-_3>$2SbQc>#b0B2Fvwdv-0RhDy1C{kIR zGwi)2MbW5ae!h4s__K?VvDH@RNxWvqhm7z+3jj@O@tYaFtH=fT zPRYVtJO?eGm12JVum;(D-wNGHcw;!xCVOxE?LxwqEUYhNbP^%T# zIAV(D%CWq26I@)vwG)WBZjgKE@P=v{9R*n5evJem7Dpsdg7%f2!C*&xPXN9*k$)g^ zV(KoSqdsTiQTqt^Q&qp0oens>R zV}04}D!x}B_33$DWeBQB?g8iN5G)H-MB%ks!Jm~zOKNqm`9b1;+tDv=AQXT%7=$HZ zjg5^(-aqreru@?K&k-NkW@6ik{x%eZ|KQ)uAu|!vIW;zc2kO7Hntsy}_E{@N3{-U1nepk8Vq{8amg^I$AXiMLPc5L#I$-fSFVBzY z1cey@K^zrayttk4#qzvd7xi@Pgt_&7fx|vJ0u%Dn)k@jepvF?%Asa1b2FZl30`EwL zDR#9jj!yI0!SOskpyhR(dq~yJ^v#2%YVHY}!t;z+uz&o1%N0)p4H!$ ztfwqwxeJA$gJ=1v?&-*x($dsxL_h5X*3x?~QIlT@=kLLagkR4Ljorh4Hq5CJd_~ou z@gV-Kh`%RqxJ7cHd4e07wItYhXJgx;FZbKNtm{4-_)6s~c)g;zqjUAn>>KTt57x%n zSw(HiWAk?fbH~EBGdvb?&M%JJQ%)V|C10q&Ib5$?#uJU^=@8!?)dU1MtxFpu1M>ncMyafsc*0SVdvb-qQY;%@%Fr zAGFUjeeIRQK3*nOusePL^{j3zsfkc2G1$4~tV(I6Y6NVbYTHfVY**EHaM<|7$V(%o z$**$p*N>#!mN~L8h$z`*DT`?K2%hFnKT1S>*Y}7#lLBlM4v1#XaTO2PVHtW{|D%pV z4{R4x(<5gB!v=}^n z?-#4`FU*zbaD&cY6d6M@a;ixivR&EJ&)MzHo^l?slWu#{TO>uKClQ`r*?b^7;6p)q zjUp={s#2#Uolct)L~pj_A9gYJ;h6np;c0*mSqv7!67`>CO5H{rK-z$gIw%dfe>PXp zHO`Qi@$(e$QUNCnc$AMOS)Ze@%C~Ls-i6>w-CpkWCk@}erFkbUJ43R1Kh@D`rMo5r zR4O@XH!i!Gb@^)I9LH7C*U+_v`b(i{DqU?N?3eM!ez_>=Q9Tnx#DgT$dF|yWP-6&2 z=x?%$v0w3P;Ty=DGmqmAp&4Eb5uKC?LI|C;lo|Mm7K?`lL_|dEojc#<^~Ccl0Bh;_ zz;G>CM+3cg^PZp2*^eu*hsBI%S>gS%^!vCD+H?Dt!v`x2m`JrZe)=_=Q0+L$O&7i# zkXONrEW6i}g{Axg(BhKF2w~3!_0=e*u3cx3hog6Tqbd}=Naf`&kJ=?PYy-JwTjCMYI!X<_)vL zXgF)QAof+}18Zad;AnauAON@a6*i|{_(D;x)N!t8G~>hpd?q&&;j|5u@VT_oo%Ihd zd(ClImIuiQXc8+Jc-*8%VojB%k3_-!&DuyTdEMNE?gQ;CBjB1nMY&hS=}o$Q--Vp+?_<`!mF<%ZAHS0 zVu5B(Gd|-J(6%_5Zor`fWVB;m##?)aQK?>=I24ESfZ%`Su6OWQDCAVbQ}?p`iug5R za=3u@ZYOR!HAFz9j565jPIdg zcCMaDtCdUNcFp;cha&N?@Lt|!D4Ej}`RpH83E`U81j8%Jd{=jE8gzl_c*tgjUj^kQ zpH7cWS19&P6ip<_kh-!Y;xHwg{Ty-Lyq|UY)}&8qg`+g}*W#^ii<5+=O%1`a4t{>@ z|1t%zTM~44zQl2kBDT^x0Hc>^9Xluj{S_Xl%Nx+}h%6b8N?`A+ia_=Y76Aj?tkqf@ zqt^awZ(+$KB9)Mj+gHg~YA_w+v9?a)$^NgW`jy(LZvujdnHjm(>3W~R>av>uwl?Q# zkEol#<^&#K2!CK4&0<_Vu)YqbG>?d}M}~6O5|vdStTdI$16AFVWWZ^t6@H0v1EpgU zadBT@t2%Yq_1MzxC%Co2{+h*2c30{87&^H0}doAD#;@anU7fmRaX@KfVXs=iB@M^6h z+rvGpF55&ivto?MPMfrX%t~~&f^cS6N|NlEosGydsHMtNzb<-cC0eA&alrO@BZJ`^ z7Bu`WbPWu{qRhx?`>m@do6RfsO!A*L$NL&o=LeL-pN#QWF-hjnu4t`YC^_BQFcuACLnZ3qaK{b6@npV#q-kuKsasxM1e8~3I) zhzjz+h)Ph0q2Bp$i45LR?OM%y?}j+|$_tA16*cizyIynKkshQzSuVA($nuVe6+9VS z=cVcSaNr-}=L&;w?4rIpx>6KG)eTk1KH)Dr4Svlhkmz5fP<}d8b$iQ$?PcD;;X7T= zql1{c;!-0{l)AUSny-tC4itw81*pBS8p65aXF2Umz)jwx(&MRnnA*_CC38=SCOiA) zB5Gb?@@mDAw(f!2YBy&KIQbnM*zS}%jgolsNxtT`Q{lH3Gp$7%J&5$X>~lD9WOLI}>)HZYHzRUKp;7-Nt&;zK_Hw z>ohbrKfd2?3E$Ra5x%_8IwYUL)z(ndX&5Or7m1j4pu3_T&mz8DpYbzJo*)gGaC6;Z zZSZ`in6Jc4P+4za)727@ryRaFO;yWzdJykLTp_+3JkH$GHpczIIlTU0BAv^-y8E1X z2A8T(FA!Hjeh7MlHJiyAleyu612s(umfNd6g3UJeN&Cc)5}&S!)Ol?VS8fPns_HC{ z3BR-KD%^pv*0{EA`bwU&v4>YrZY>v$6GcKc1rS|4RfWs~4t6rG?}NYnT;bTCP3-6& z;^#~ziOF8OC*8XYsjt4yOgGOlZ(U8$ES*xjhmDjZHH1Xm$0d6$ekJlY&GIKx0x{j$ zTFYF~YzqOWNTM5d9ZV0E(kTX)86>g;Z|S)Pu|^=e{DLJTQN7Hp@K^=~IWl4Os-)>d zWL2zVjw=bflYjYD?2XXU-epiTM8e6gBDFwE1H4q`Th6Zrz8O7j(OepYh% z^|D|}HtG^?)R_9f`vq1>aba2Ja;4{nhf1D2HNjf`A`%Eju4749+vblliXZ=U+Plq)0B)@5WO?j&j6wQy!GsWA79FB8Uj_enY)(R%3dF^jCc z&(&!zuO5!}LQCC3a*3Q_T$mcYU zUZUa_T^eO939H>-`DSO76*9${Ev0T@{AD3_O%he)3L0g5Scu|&{i5#Xt}s-p>DPRs z&xHVP5#6(xwNV^R^k~-q)SBbHZ_eE6M9yk6uP`5XexBER zY2)ol@Nv50)^NlIjc#gH?n8D#CKlJutk!w&H|?fvV31kOOh!_ABB4a1LT7uDm$uW0 zx9XCPg|2MF+)EAx!3Tj6pI)DSOCxK1d~njc$X+VvT{Gg7Y(sELz5kIZ_woEHu~RKa zUXGR5YRQ&Rz5g`HzKPUYrDsGena)f#3(vdR>ng=LNAO9PgHJ)DG2vN}9W{Yrb3Q*d z-5YSvBqCEPijE954SL8p;@|-shZkORc-M98_fvDjF7i!$7rU7W5}T-n-epUQAEU*w z*eE>Hba!+R!b&6#8YDFI7LL}0Jyb=V-fD*|Q5uCnf~ zdLD<|6@6bwha=3S4c>ogoGZk(Vl}a7w>Z+LgbYLHm>`Un8?J18c9Z4{&h}O{!dsKa z=~!EAC~v^~m&Q-8o=RRfw{A66ofgP3Lg{lo6E%Ni*x+8(8EMQ?Ss&=e*eR~WC$&d6 z&2BuY1vh={WIe@RNwqr?dPjOVPD=M(|1Thf$8(+!mzSr$b>BRz#Nbo}2Xq3DtQx?vvVM zGdpc%ye@ zr8lEHI5Ej`dgO zd0iE^9r}#C-g-|scWvc+v)g);u?WwP&n{kSc}q#D9KY%rP{h>%BxItx1UYu=1+X=;fY~T5+`*(n=m5aqEpf0_9H8mpD>{m-&cu^ z3N`EMNE}YF+JB{$?Wg|^0i|?XS*8~vAc`-k3&C(B@(MU_WV6~B3ic-810H<_2EK-S z`&dp{ge_vx^JGNC(3g|uZc*k|t^&P*k+-w6+YwsF;+URyH(S4yP z)fMOvI791`DMqTVBj(g)`ZmYv%)-#!V2``U_Ni3Z)&$w4nymp(7Hoc%>bl!yBn9!H z-n2__e@e_wBlnZEON!Byrr=Gyx);N@r2h%idPQ*V7BQ}+F;y#}XZ03ib-P3XPyY$Y zZzu0m-!u!7(_o-I#e8NbeC|I5XRzlI?B^ys+Prdp?=Af1iYJ6TM%yQuF->z~tV)Q3 zJ@YJokmep|{4l8|h%xz}d?I#GUFfbP6sha8yu)7b%`gcD5JTcB7=C$Pulx+Y^#b*M zkzB(eyJj|}!5lO|f%qS_;L!()!<9U|>jd>|-tUrB2(Ob;XT^$7x7;@Vz2s^bq zNNr?(#x1x%i$D3p`@47XN*xa0_mP=$+86Fdkp_lF;F|c)+%^7sbZ5&;DUm4V=?>+v zTTF`P2Q|Y-(g?@Xq|aWDVQV59p-$}8tYjn5nb6oix3xso6!~sRaaVav)yi)DTqjj< zxF;nVB>980ovL>6t2g%IESW=4%`jdIMIa`FwRZ-;8%*T7e88pp+^YokKv$a(v}hIL z52WxXQBbJ8^zz+7#m*sF>q|ws~J(X;g%51PyW`se^cF87V zmFWo}B%Ja3TT*LXqmIWxRlN4(W~WTq+e@P@y;(dGFlu@$QhlC;f#ah~{$7oqD#jYq zplbikSj+spGv{WKS`xM4=#(5fEi#@Is{lJ{q@E2!&wskOpj)24rKdhxtuc19-hSpR z=mHW30{V7Jc||?JW^lz8cgM5RTBSCXqEwfSJ_V5PUt6P6@Q7W(qqMCgT6%bX;txZ! zOl1s;jIG>D_JT(iLpQc#KMlpbT#1UVa(P>a;+iu8rS05#kb43D#yIkfP#4}tTjaS2 z_b0wZGq77&+Ak_l9J%W#PRdCqBaX8C2Hy5e^%lQlK^))wrDe)F?1f)Nr~Gg$>P!OJ zq_7C-qLTkB+W6$%{z@VN%lm>%@P3n*gOxTnD?L9o^K;|tR_4S+AP6cQF(k!n106vw zEDXFa?j_W?)0HxhwCpp}>vFT8Ie6&DpEC(%VzT3{i$S$R!6jDuvp&;9;`QjK0f)kv6>1CK^tM zNrm-TTc69T%NZ+^eM2uZXqFnzf`Poe=#k&1#5;0XnFUGl4wf_ldOMvl$o?9>`B}Hm zMg%lJQ?MZdp@8`(t9uVjoWp+Xg-%oq(dPWOg?u@i%>X#s13SK9O`og(t>GA9q1%vMthn^tj2()au-WeLJu;@$MBEStNe_!xDNF~N9S862UZCD{O?enG0c(}XxkM1(& z$iwN|=Tsh!7$OW8rM9@>ge>SA%*-U97}KshJPl)}~Xif=sj+Mus!ANIZ>? ztFpTx;$l&+wI@F?dsY&5yn%xc4a`OteUI_w`_6O%E zey{vVvR_6jv({3XJ05xQLWEG4LLcfq&+{H>Q|)cyrK2?-p7wmwm8eA znW&!}48hxpAaRbOahO76X|%kRgjuM=5sZJc0elfs{ep%Z7#GdDO0crrd>-*IG?vLY zDITzf+8Os{uL9Sx8r@2y>BzZg#*7kjSvAWa&^CRSG1}3xn_4?V7gl7~OmWLs%xK@f zSL~-^7Bpp9xV2l?$MxBwi;?)47(LK<4?+pQH+9FTxCEFfh;+anXYAsQy%x}c{5k)~ zf4m$u{=$F#s2BwY7Z0SaVX3WBh}*Z%>I7#MLck`(pk)T?Fw9yUb6>%O_zz*7py+fb*tuccLJ>&woI8Yq*y|=E0Z2O2nY(~ zH=d!*taU^nUaPbjNS)Qy;hg1VpJ#3@1X5jrsfei9*n4<*C9eC6ZilO(At6D@k9ht$ z%rlLe6q9I{C%xwm>Ib}TE3p))&L5Z{9AUN~Kx$zn*$b)uny$T2A&M$Nz$N&BWxT!c zE`N2%zDDhFExC2x${dH`?S#_r3zxVlBi@nu+hwh`OK&>0TBg*Ak={HA3JaqoB~3p8 z3tzgk3rkA&rxiP>yNwd2!A*5NX5AX1EeSvGob!38|4N1iNa7oi1VSh9EsAD8ue{+5sN*#HfgyX>g_ZUcskUp! zcLP-7A07&#li!ymT^y}&ihc=410FsVm1Y_cfEUw`)hw-ebvUI49a9+V)V4|T3$nqG0P-{o zLIc+=GBT1xMWq`5zSp^$E)z!m_{oze2RZM8gI@{e|NfmTu=8*sZVD()q^%Id?ifyj ze0HWkQ#f3L1BcsNon<`!Dc4wpdzsV4D=*rFp`HC6v4t%1J{iZbaPt^DF8m9u`s(dT!37GYj6pN^mx}wx~R| zOWqN)4+om`C&E%{fBpJpQAO(K2PW8Vm|@6teSx?29u~xlt`*R8S;9|MN119&yH2m( zXn6QejpvAvrbAo1<)7V9q0Y}_))nqJPB2X)$*;%${@}N1a&mm=R*y>o$pz2gTGSyw zA$T#)^_?A^V!a!u#S{XLpHoxy7Fq&uA72y~f8RwTud#F>kr5XWyORZ@4?7fmwpDM& z`IVh?5v+hgUJ*3}&o0Oiik4pJc+ zz}MEVFHefJYYv7#$!`-G*e!!$Nsw${^2Z|96={7ipL|nxkKBon>HI#Zb1r@x@*ze& znGKl~cT_6f8iocmOA69~SU7jUQ8^Ca@1#xk~707|ER{}Hm z1K<8WWGGH58C!|Pw5HivM&i_B>);OigI1tS=t?6(-yeOja$-1Ur7~DjWPQ`M&XTrs zE#rgRqF8o*+t?~_U*XsH1;O6dq?V4>oE`+eF8(93W{n7h42oG0D_@w!=Rz3>zxK|eNftg#dI#;r^uW7f0&4+UHKkUy zF#15EdPv03aXVNa74I;pL|E!YvG;@H@MW(%+4Y4zFSD61eAg6aMcx^o;Fd3h4??5` z!&_S*gZ%K!dWBhr#s^-GAP6ZPHhG{jzLJ;J`=D;d>Le z$5j$<_+Vk0e+;*Wx1|t;Dc=T{?T;ALIo{Y%aa!WH(n}%zO!j1#yi$+ED@Ub-coL6~ zxr+qvjMTsnSd=Z;M*1O@7S7kN>q*Fz2~lLJA;@%e#*X}-babfyxCZ2gsfNyA8xJLL z&M?7YDAz!hIXcykU`O4{QRWdt`)F;re~kWbSI=3!a8T>Wp~Ej75M<-%59Z%3`a05v zLcM$xO|tH7iFJz?7I~LjROk70J|0S=NnB0j*^Il|yN=tk;lyEm0=TV^=-M%gw zT8q5yx$Tp~etk7PK{;c}KW25UCtyPOE57DNik4IzDFy32G(EvT&XCSaC&|t3{mPV@ zpeXNaI9iltE>J{Fq@2T$a`q;y02x3j#qh=i38)b!pQ&^^KL}aE{`x~kJn9Q%1X`QC zYuojtAOp+I8r9?42tHc8FabzPIQRJ#sKT@bIO3(tS3aC~l)YbWEd222Ye-~W5yDeSF-7vtjvTh40F@MUF6e>)?Ru21YU%bV*z0^fLD zGXcn93mL&H!yyaq=F+#_M+0=c`6hYJekh>T*a%UG>{>WqQCUrTwg!L82oNj0mqh&G zEQ{v|rmXz@`q&4~osC5+N6&4)U;Tp?ZF$Z7_)_KNK|v0;3p^=jl6Lo&EoIfrBYu!2 z=WJTq^aWu!wl2Vc_wUYfpIVJXI^(Q(gsri|GxtTiA2Me>x zG05No?BF&{%U8~2-hsyK659q7Rx_FP=+3OGo$K(c*xQfCsqctK^G-~N?5@A#F&C#A z-oC^wYxMa(sMwlgWA1!SW>aZaWY6!zPvKb6EyQ`xHKjPL_i3+K-`sUuJSP&`1+IZp zmHtBs=VL)Z0n5}UL-UHVcYv9ZoZ1%Jdo0HQRmkKVL*^x?;!j(Whd)>bI;xa_vp)MB z^=?-09Msl7fPzHJhC+*PiHH1Pgi#XauQ4HBNgQ)=@5fntwEO2((WaT(cfJ{Z7B&=! z$%1&OyRN*)ZjaZQc(3aXuXoj5I&Dpxe2i7}kRu*};o#sZAZ_dz9oKXFWnod{^|{Xw zp*N!?%6FmP)lJ)-q@9A({b$*709A=WfQt|DR%YOV>3-TwRorA2hRe>a8Flrduicuwe7^{36@q)G@|sQ{<)rnUHFw1rhCbz-VV}{y2Xf|! z9WxAydk57%iQsvkjjVKZ({CApf&>fHfB-J&S2sd`hVh#A?=wJg6j^@1iuAX!^mg>{ zOaS5CH}oLYzqbGoCQJDR%u4a+<{DZra{oN#JECVREcgPyt4JSSHhIpjJw?9tw6!q- zOmVqrYs$$X7r0FQjnn_mn28mg2d~S>o333)% zeVK;SgKaVw*5}yNnA>T{Gh35)a!-sKQbG!goOgfd1prV}&bY@Yp$F8B_@#6Zj)Xpw zx^p7LeSQzUAkFb{x@4>%Y0^c~byJMWu?9@+KECHKqJ{Pkm)XBAL%#f!CH^oNV$H=t!t`p#f#5s{-p3d^GmCjw%f6%K-;U^xpi>aRKU7!pI8ui*~&6xAfo( zx`Qi=b!1jp@akL}V6qX)4i1MDE}+*`P_;W<@otB#rwuHfy$N356o6Q4bceA%`8cTJ zDsV-qmfvbNqWH4&!oTyEJQv)cS}}sSG!;p7KS$ zdWqx4b6i~9N)ML1Afy5xMx7Qs0u1Zqq}Mp5z?fb z-w+>S(kt6D{cCfVZqcaA5;wd3@ALb#Y#M&fIt*+$*1F8Ot$`i2hIO5sU!0<8y`LaGFi z1v;HSr|-HY#H2mh|8$9AR25kOeobHbfU8>MR|Yk!rN`36?meJvzLffci>D++O?h%i%kZ zw~3RcGQ-5f{eK7cEHV(i7U6x%F;aJ-X-_e{15x_@euD^^!xLk1#KaoONgdABSl|l$ zmX{6PA@r4o-T2Oe%v>F{mJYngUpXoYw^}((esxLZi@!{APfYvX;M^)nCRb=N#aG%L zWW0;mKt!e-bBz8PQa~#+KuHAs4p79%w66priwu&8rVRb}FJ5~61bb+OJ&sxGlA<}- zXJ$c9DPl0ztP8^=OmO5P^+~1>P^xkemwx_zCNOcwah!1mbX%U^QH};SK}jjbR^AoI zQCK0O#QdVr^W>Y$)Od+HSUU$p=4D&RB-kqci;3S|1B-;Vo&Jn;P#@*|d%ezoyP+Ut z{(cy^x#cR)Q+utfZjc1AfRL`DsioCDKRR3E&>l)3HuJ*$!0uYzE<@P~=gzmkrb@7p z^ZHl4!*j~Z_Xq{{OxS-vl$Ds0vebNx$=;qE5ATaJ3SVe;_L@aeLq}!UEi;U85Z=Uv zWtMH;dZzmm{mCGIhO?U1P)`pAk8&#_V%-0+QhlI||A3Oi_6rLzLFa91hR)?C%iFJS z0!@C8c&nF=Umop{V-b(@-xB--F(Xl<`)j#>YDw6ie~*Rx zb0shf)KdNk&fnT>a298%&7|-?lCku%c|4OQ6B+Y#1-u}y{i6Y%H&(yfSAvYdLB3DS zyMtSDl?xRCFcpxRh=9Vyx4aSdV7d8NW#=6=rA)54i@eR%bIAmxaox{^fm}c6+kYNl zhxiS$iV%}g>OTr4|MxMAAW4lAx7hSypUACMY3eVgVe~Pw7oJH|iTqtP#aYXUZ-am* zLOCfb3qmSatU|}v3rp}AWbTObN4&IAeOc|wRQ&j4FD81emlZ&ZcsX#~@wk{m4zvi1 z%l|P7Xx;DDlk;<+3XAs(d4&vnZxP=h0AB9*B{}~Dm$ZGXNM6;nz067SE*l?#$5wN7 zVdddF%BhHOY59+e{uXdj6i|4+VEH?^unEoOi+eFzsUo0D4s!whRdx{7Zn?l+eAvvz%v%VbTn3G z27>8-fCOg#uV4|i;dfKh88J*O zL8j)0wXZDy=a!i!z{L#1QWgHZ$1MRxB=f?iD1qRebRXC7l|No~n zXY_n*9IO;y)Ws`p(~HOl>LuwsjVJl|mdDz9ceX}Gyje73#L&8Wp>ND$g*{m=$2uNo z?}slZNuR;B0RsR@J+Ndj{#WsM%BC#g_qsxNo-({j($RK&lMa@L7f-#`()VFUQIyQ|*8=X7sgdhSs{NLJhQ&|#w`;{vO z%IkL}I5=^q&4SEXpIn)Sxbk0;Cgl{Gao+j(v|x7`JftBm(?qc2iFj8>W|GkR9TZv6 znkGXwK=c^sKz;d8Rg6^?2lobwEs5j&)jN%{y_vfYeD+l~TfPrcc!&f4(D4QC|BI#b zp>ZvTF{xN*qImcp`nwN z7mClCwC*AT%PVuue0fn&)V0b@huDq#8h2(c8wo2cQ?%J|?)1%(37I~f`Wy(@732!y zzwGFH)U#oZ@ivYOcnG7vYA060UP%T(V9=$whdq%6YfVk zpfG{W`_x&TCeUrjI&zhY%0;kmT<%^@Tr|zNE@*%FfhFN!6*@4H@}BK+_&~Q;+BZXZ z7W(;)&dL8n)>{T-xjg^F7t%;eOQVQ%BOslUN-GUF5+W$lskC%=NlJH@0!o8)gLHQ& z{p^kBdpy7YnlPdFc>wyP8vdXjLp z{(8#c>O~l?8!eylSo`j4{aO&_Wa%EOKX@F|NM3hXKm4;5NMmx%J`fSwi)Ff94r-ErTT@Wb|19wR5!s~m!nvwe~a=70WO0P8C^ zDgXaffIH*O?&>OEJ`K3MJeesc@loV zFBht&uHUd*lC5^^@1p`~H`L(oj#8C)SON`;))@vKcsH zt4QkH9T-SZx8R1e^?UENz$@v39ZAxOvZf6rw~pYTFu&(A5*?R^b{UdDpF7}KR=Zyh+k+OJx+)>zV|EV3NlT))mp&YxApITm@I#CGu`kY)A6?Sy7spk`k6 zfbK<~Qvb(-N4Q@T>cNhw30hOme|o)YS-d83b}Cz~NE?D(Ww)4fg4`qB=bfdvPEG|f zBZ}Dfs{iBuO~NY-o&3x~x5bd{pxazPeP+z*$u!}1<<#b(avRI=`k>Llxh2204Aq9O zFQ|TY3~*eEVsUCMkp?Dug*S;=3ac-bemF6#RC-{AFgXTs1doM^E!ea4%DM{MWU z#*uvbAm2(<=~Uk7d*zP4Ii`4s~SW74U{ye@8~&bxIuOt;GtmDEwKE)y=4`8`&+t#eVtcVbPl zlRA3+$Lo`)iEkx%-4&i`GI~Dqu`GS5M&j<6`@1aBPHu7fTTq$;9v)r`b(2&eQTR;d zM=rmSVUZoGDIV@*I_%;4gr3tY1oI(Wf#=envlx}YtaZn6vOWt$tgkh-?01VJ-~(q)i=&WO(3El)U*6>Knp?m7M0X4 zV-!RSGi)=6f$f{mS2|#pJr4;Rzi_h$z3Dt&wLZjD0jq~L*mMqs{Zyqllnx4eZ;;Sa zK}~$Pl$Y`v`7XE6IGOlv7_0rOH}s$~XJf4W73FWVFUnh3 zSh$p#L{E1o^(!LuMGd`^bE0==ywtd1-@!Jdr=plnLKj>YZSGZNA8tgc5Qe`IBjqqG z!^9F><0jqjs;9;Qi|X`Gfdt4uF(I0QprI}BzzQRdx$SX1kLP&Pbn&B0EB6j=nGVVR z4`?ycmRt7dbH#l%Wbq5JAzWua?y{*5k@5djKT8n)B0eAp;xk;Ram#veO_I=Nui)pN zJRdpSLKj_W=|`5d3nptlh8-N-UPZG6?aFm zfd^rh3!qq2ZCif`K{YAB9g}VAx2Q(3%lK9+M56OM*_+rj$>nr9E=5P)onplMWQNvg zn9yVUmA3u_6uSPGhTR5LsX~!R@+B8qzDRg96y_PeSVgSC9N23lnBBR?9beIsX7Om^kz3_WdU1cU)7 zy={9FZnRS5d~#J<5U??LL$GWLqKV4?EaJ0vld)#*S27j^E07zkoNDy`q?0jnWr|WM zj#7#|rRGr18iG0IurSc!(szdmT2gf&cXy5dJ#K(dhhMO^+UQWH5}zp%LeM>9^7*Dd z7S%p-(sWa;>#@(a$4Zf%f@4=|qK~H-D}SnH_uF7vGIgN&sBL@wdq$G{XLSPajA$|N zjgRUKt8gI5U<07KxT}QM?&K!%b1tzYudO9C@;UxU8|9}~(tV0TfSe_<_4kk^`TrFA zaBiEy!3Kbw><*A2-rjNHeZ)~z6u+G)Ox9><_hMiU|8N}p_>Bd+`cnds^^d}i2M~_` z-J%0H$H!cl;WL8B^0$Gad7asw;iw*rWfPZD3~hkaW;k6bPqL7upEF1TQZZ1#5%>Ro z>p5bN0Eyz$_mOL;HnZP*#{UlXo9S3plYTY7+MRLt0@*~A|EWM5_Fw&rLizJ6kIHL5 zR~j42Mx>|X>RleIJ~U-xKn&pgDRd`GA{$i0{&(NHb!=z1@=eO9QWt_syy91Hse7@4Ck0~#++*dQvdWsT-2tLL5&!?bN<Cw`p9;Y8{st%vk=Z$de7v-WX6#+LfEn8gN&FSrde7v+c=#Iw*==cI zBB+LZUu_0U&QAfrJK*4ZcwNk7;nB`T{1CtrY~!fgU8-TgKUau}2SwEX9jDw&PEzG)qEAVTz?fOKS@xM(xb=u=d{`ETAF9Yx zr%N?TKsgD0_xOK)vE|Rd?|}mrzRHBO{nlWxr3`pAtPIj5H5_1r1?8{k{{9GtpG_VJ z5;(rG=E~P9fEB?$I8^0%!J@$7{Ev)-0h>GktKxW!1VJSKidXzQm_J_M^fcd_zWa=! z>E3yQtuWHNHyMi?zwu1?g%S@FCuq2#pF)KoD>x077P=%6nd1!Jw7Zc8+UjV408;vH z_D724pT38_PG-j<-F*liOS{RWrDIeiS-=tB?+|ak{}^-o`I|>CUl}4#p(?#;@-P@p zF{mP(MgiY9 z=%UGYc?mN?C!!&22v}1%4$nfYJNKnG>S#=Ci5mp{e3{8eyrB8aCD~6?NB5nEsZacg+SxH%SD;iXr1!Fc+q)vnz9$OdA1ur> zyeL&<2+|w0p}C5-1069~q>4v0wTA#bEhL{;JcA)^C`S4PM#`Zyq~PbrezP}Oy4rA4 zj%~<*>JRs58FDZnXfWnBP3$kQLa@JF($#KL7+X0Tj<_p5gS)n&l-%X1`a|UAeT?&E zB@e^;vJ?5{Umq_fh*Bq9yhQpL=Ov5TVcdlZea&_bKZ7V_gq2v>ZIXU0>$-1zkO&8K z9p?Oi9h8X+J{uS~Hg_I)+`Oj0jX}LDHXZ#Y+v&OGDChuKLZ18BP2|vte!B|_jW#LD zVw*Oga1Dz04VplMLjGj0H^jSh>Hh%?06uQ?JoYdYJqN&h&h{=JG;yVrO8~iVkqDmM zYgL@Z%VE{rx&nLDcR_)l+RDWuQ{-Xs;oin@$Y_zk%!GU$;~Cjmbn|Q8>ql-F zro144SUIEc0|4-IiKn;L5Ar`}=`LZPKA#F?TlHfcDa&SK; zkJ5R}eU_fLSW=w5TRoKyr`mUwUD#?I(r70Dq*&Kpwcl@iKdNH*y*r51^~bY&arB7P zAiWDRb?E{C2n)hhPj9mx;O^h>-K1y-_iboA{Z+A+Xq!#7WOf`du~|OKrB*(WH6^3L z!J;EyKLa%71hn3j+MJVVC; z45!luF`zO^Tkm0DQ{6ftqBG*eN^;E>PUPXWY5g1ypHs1H*BxiCy#6rhvahMF5q!(h ze8(2vMan9-JsM7bDmNWEu?e@AXTErM|9&_rQ4|6KeYgMfVglo2UoEi3#XpHaGScI- zJN%>{Ww5LG^eKBVA*iVLf8O`&udcQ`Pwu4%IRMQxpUNu8uJSx2ZsQS7aOTNm+OW+q zJM3O@^bxsZYnRrR7v)Li=oru3-{_{im&jr4eGKn68|ip?c$wbZ!DxQ>2j|_eoYmCk zlE%*q{f3<)(@PXj)gRcJANdaZ#su?cH4Blvy$op`iIr~8>%hFPCMH_6{nn7JB!$^H z*)2@(a&#@mWn@(^N@8Qp_kwk59noE(GbsO>n9#{>h?RrW&hk^y{zAk4!t* zG5h}1nS<6bgq>oxf1dIgbNb|RG?;wuXq=rtx)MTdSqc2^yTo1HKEiVtr)-A5OW&5S_oYDn9)+rYac@V znA>Gs_kv?Ki{ za@yx-DOxWq*LhuB*ra`18OBrT7V}JjcPdI#Y0|~lPUP(5=i_FxgO^QR4--!>!>`9( z*VIgL)&ufi(I>iro+P`v@P@u-50|}=Mw$f#=CZGzoYcOZoJQQcAW~#EN2~QR=Y;w| z=@(QLhF_Hw|I&Iv3-`h`yz;eSqds2;L06Hh#hRn4?Q0D@DgtPB6t^t_ zjbmV;pPUB^UTWqLV(j`o{&gJJL*?cYW*bgbQMo6%ySF@8$XSnlyX!q#?G>?R*PC~? zWkKx3z93J-OETzmtMS~Y<-i+*P zFj4?N2GRLh3)V%YxxZboI`lvH!4ui(m{-B@H`t2KSQeiR?w@#}_4VXK(Bp$HA?8SD z2x+N{s$2xgwhWUc2XvQ$EquyEgWw5ixbIBEr{5z)Dg}EL-9t@H$iyO#2 z6({`e{UXl@Y@HhkZxDAg%N{{g{7+39ndC;5Z*ZVm7dX%t!XI6^48-ZU44*KJLGqd7 zg}mJtmuy^5cBb81?<@Z3|Esmljh?_jW~Gl~o4)Yt$Wy|b$`vtr$ixdrkn^2bhNJ7K z!OgbD<}!4i<`$U;n*7F~Ip>x%M2&Z02d~w<0>#aL*Zzf&wIiH#`uBiz+mfK)z8|>$SLoD1X@5)H#{}QI2Jpwf**p?`{7aLf||cw5ebd#s)w0&$1xCFY3x|Fws*6 zZHdX}67{~emZ|xo=;|fN7P$cfotz|Wh(9g=ocb|1^^Jv#qrERDD=6^mS5$jje5=LA zh&q4t+Q|A`Q11b&!*Xr>A5`iso!gaa-#V-jE7c&_9~IY$xC!+b1qCn?An)1q>2#wV*Y3M@_+6OAFoh3^id$u zPPG^O(KTidQxYcMq-?)o0aqGJm__IJJzOa4&L0qlx^&}}q>}VPxK|t-B1lLv7ADCZ ztshx`GqLVr9f_A77asG(&odBI0Ob;pMTTEYh(|#^`_7$bVI&!F;08QDH>|}U4U#hE zSS+FBAprFTzQWhCTLF;&DnbKtu*WEDWN6~gy@ptNwHx+t#K~R=-{TtDpwOMDFvk`0 zsd<}{G)x8b!mI!j6NG<~4{ep^sNDC>pMaiiIIfxsTqf>BYbf2XKw0ct$cyegWfhU} zl|fgJtD9fSf>&QauMga0>fCLAD?}`Kus|EAx^Tl0^kHrO5%!j>(bVf;O|7voAdgZ& z9#wtd%FFxb7?)7~JHcYb-z>gpC3SuLsdLI=W{6JP|w$?uCDYBqJt}gUHc}2b9IFfY8 zl!n%)?_IfcFW-Tt^EpSK@ar-+R*d^kAKXTP%S{e=U(}6)Jp%;UgJA&>HR6pOTM+&OoCTE_oTJ=bNpeVZo0lRf$^-274iacZ>=sP>lht6U>cW_tuKqi~!K6|mxu#^UfO9skL-@*(m;-Qe=2$rDHD!KM$Ay%Y2ORP}Ea{uzu(; zJZ%oPAiAXnZc{`07k9wrkG1^-kz*omMo|!$Mz~?7AzkSXYqsC7C03M1w+E@FF)8A3 zC)RtVC7}Od6;>F-|FH+1bcV1zF{Ic*>ss1U!}4{cc5nD69boKi?EA?5%`d)Mq4s$L zNuMTCd_w*hR~hrq(hpX9V`)52Tjm~Ldd2c00n3kkV|mX2im^gdDXY`pvEQ=$QMhoj zySeQOQ~0_SiO(LR{?qSgqB>g@ZztI~;M&lz#)M(c1v*;ch5nQz4t?K?Wce!@Y$#3o zWr{V`kEXk+T-(q5;~Yz%TCT5zWrp7Jugt(@cqQH5fBPK{Bg>~ri-*{rh^01f5>~*S z^8GvwB+6lajD%{W{l{`t-`{qP_xDFjaz}vnVi=ZM-k<#A%;#(~G`q#=IahX^+boM3 z#(CPA9`A4QSM>{huJC5$&(FTrZa){fJtPkBQ}Wxl-pAEszNf2y^=GernF(xknCNep zcEKZZJm79Ug~0^MBBOTv6|d|l_t9>ZA(}e7*qv1`md}*ype=`M7S-YBEnI(!OUq*b z$T`r_1P*K`n6#_iehS6BgW$-)p-hqJPA1Kw5VDT%y+af4c|Giq<6F=hgntqbTD8yo z-zNn;PxUj$`c5GgZKciXt)h#rJ}-!_&Diq(0syc!Zu6BCLO)M{v}5DbTvyU{NRt`m zU+wf+&$>SBa}b&0a3>Rg0c!6x_^7MhXr>1K;t;e(%h`|e;QI(qxm&=!B)BK%L)Cr3Ec~ThPA7sTTJ`fZS({O7S<;8js+ozi8MLUqS)e z+|qJo)W!PZwn0VJ{2re|%v;ox-dH@Pw6`e0Apwbf!0IBULOGS=XzSsA*;_5_!eD7^ zpDPH4HjHS9M^pN9h)JIUOdj0weW%mWQZ)An;Rc=$!=}mUFzY>wKF79kHqGK|%%ZN4 zQ_c#V4mCOj2#Osp=5tNfK6z=W0k`R@UhhlxQiabKu9>f0n)|+-)WWL-m6DkBMP|)1+^d|wATQqQzwpdHD~ys?0qws* zKh!8*$P>ah_xR}i<jOGRrJ}$PT7;F}{rhSh;;LCkK6im6l@)@YRd~2A4HmkbNltkc!!_Y&N9JkH zxziH^c(}$Pt%~?QQFWb`)98;UNwn!UUmnhNXIgfK6IdH9X9)Zjh?tO{V7FEEcuJCS zn$(uc#J{CdAft^U#^Z&mTi)u7`BXL3WPn<}^9=ixn~4FMMHK>A_L!DaE%iRebMkoU z9kgI8_xSB_O(iSK{1&wCDR2}hphM)?2Xv7`@0rYeEVZ%QWF@kLd?^;-)l;}E4#B3$ z+uvf15Uv{{1oNm{&P6vXV`3sed^smlFc1NRq<4Lj8Dh2YYumPU^q-QR&13i^rxgC0 zN(M^~1b84ZXqZGlWr%FL0}_eAcr)_J|6}x0z`pXU7k04P4uQ@;pOhBg^Qns7F_i zdYZN9uvh8roa>yc8?s30v=qMoeUDAQDT(qmfJu%vH3?VUg)9xuBMbqoN-QuF-WM%Q zkV8{(dkFTPe8ZX5e(9SOQBE7zp~)ImP7Si?3lkF)z8Tv)mzn504TpU@J686p;agi< zq>D?obaZszh(I$`v7aC6l#mpYM!_TSa#-!V-+S|k)=lwA+sD6t^{0w>I64N!n*=2# zCGo#o3DY!~TwGk#cbrE)5RSKhN+0`NlK#d#dg@s!JL9iJs67*p-XWaOc#5McUxn}M z7uGr|^lO?yn>W_L#i^)G-w9Epg)w;w8PI3Cqr(r>p^P{Id8SI5T38LVtkY^+M`Hb)<%y4sv< zO~}VEKV)WxRYP%M!Kg-=G$#fiGcp8J1_1$tJ4+s)t1dP^c=xG)U(#opPkj5F?#+nZRwPE#D#6Ia)Y#BaWjP@(tXAB7b;cTaVsEcg5~q=$OpWzv zO)NO+Ng0J|DIhUckL=RZey+(4q+ZOSS`W~_mXnoLni%4PB2-#hy72w`cj>{n#6%`5 zTVIpzm>I{#V118EI~Z)K#pkwZZ^FpHz*j3IcHz6Xd1$dpBOOOb-ja!<`og`7GE~hy zJ_bgmrsA(n%}$GIKLz+o$v(%9)$QSjiS#>JnVAX+zCH0=ekn@)3&{?%ryKbLrFMra zy)HjLpr&VJ7Wl7q-8y}hU4W~r zYl_nfsg3FJ#>kRcz156>jR!cYNOx_e@WT&fWt`cig@wTlPNt#5>}0{k4F_^YMzn6@ zv$IO>4E#Uz>j}R|S|(KnaJMM|Lm*Dp(vyaec4%UAQ`}l}jVC3#qyOfX?EME1{9I@A z^7G?QqM`_FQaAun^}9{y0I9vXO_5v}mX!3kRfTA2!ZfvUyJ||$ao(q(fO+L3%!Q53 zp5(&kHnB#5);^Hm2#LNB3PE~`iF*6($!I3c1chuc9eTg8diLPxsHeC0ezAfJ5TuRm zZN(R$9WgN{)%~ox`Lb}rl73rPR~KN1(>=s8w-RW{Q z)kBz_{i)epwe{STb@R;lxa{m?{ftfdxT7Lf*e!t^M&-OlV{_qrx*V&V^hk`X zlOxvhqVqv2vtC17g~-`j#;86Qxv*=TEk`hl*Voq8#r<}w=EFWgk@KzcCpr0&Qc~f@ z47C#Oxcrmgape3Z!j4e--GJ0RYT4CwT)sKicgo74d!Ar$YlW_;p4b}2E-bC2B37PaqVM;ZwSNe0sHGwsbi+siOcy_9KuFU;PFA7~Q5s?hS?mL2Dex$@ zwDlIM=7?HsKT*{#wuc7ZJv=;ovY2{xl%rd3U#Q=F1?F+zbC7iba$x>oRbhU9eu-cs zT!yk199+kG;O|<1l=rKyuCDn8<7Ck)`3ZW2*%%n8b(d@{z=MqoSOsnXqYN}sfWMX| z$2pPrHE2_}b9KGAqx@LR1^&XK^=!lWZnFpIV+IzA&`DmrHga%K<$#U0o!7?3>3G@k zYB*OFyf{f$EFGAL3lQNI`JX<0xM1oCrM9+S{2shC;_2mOPR#!RE@2&a>33<^*~4tn z^(8(;)@;yn8ka5K>p6y7ipBtAZ84vJnjc5k4R{kf{VaeTSoq%8*9ZUDq4|6Uh+D^p z-4vG{8X2uYC2R}WA{fg|C1A_I!xODUrl_PuBob>I-WplsQ;oSF$7PDmuO=lU6KQM0ohyAw@pjI@^!a#d&$ogW?Szc!2e=X_1^A zfuB+zMJ)4kd|;CM@q(r4PCu2K_7@cv-p4sSIM`{vzWVg(Rng-rn9{*NMJeb?}dg1x;rvz&qFA%Z(4l=ztVWIyhzHTnmE6w2p3 z7e@Bk7zXd^ZE#u)&E6KJF2x>OMg*0^to#In#Y9MV`K;~*_xAQ4(VPws;Qw5DBroi} z6em9=BP|VlJKl>OHBm(5xAOY>R8E!N-bl0fxDCJ`SI;_cwjs+YB@0Nohe>D9C@Ux` zHUdMjxw#2!(i5f-WoO*S?iY_HeqNj&E=a$4Gvln)h4cV%s_J#3A?a-is)#u+$BPxD zj(Q9%tou0#OZsF)MBhQ!@jV6+ zq+=R_o8Zt{#M8x^Kgm6LGG91|yYQ>|T8Qg`Pf|F$xxb%Z`>y*hK{mExO+6={$Ctnm z6n~D(q?A-q8AQZ+jEjf2zmZ=Yam}h#VgPjINdBG;t(0Ed++3<#NC$W;SF>2Jm)m{+ z8}VqC`bS_lS)Z3J0d0D)n!v?;f5$tVK2+#r{9rjY(9o+q{?WVNxz6frH{p_`=(J;C zhgLaIjvF@$@<&`r4QjxXzU)n?wA<8pdz*iUS-VQX@%K`s@?ZiG6meJZ?qbzF{rwg`)JRICb*kH3 zIm|RPAGq}z9G^{hC_8eNxL=;Z6+bVq2?R%x`qWOVOR-)~uMMcQEnd=LmFoW)bZ7h6 z|4l29@jJ zrb2sP5JA8p%&(cjBTTRH;)PtFb(Qs8lk@S$^t6u6^~=C?s=zbhN9pXJ66aa|C%LnsD=Dp|%r~!ah!m|#T9xEo z8|Sc49<47vf_2?o!@G=$Q+vKsAHRzu+kJMtx$YL($!*$44uS+3r>Af~6!~L&t~oXL zuDCJU;P1&-#mK0r%8zAyX=PwG-ZPCZzZ#ql%}nkbpaAVPpA=jdD=8^?<2y1Qwzg)r zqV^%-Cx}irulM{!{HrQE50<+-pUAqo2~8mwaA_}CXVPGI1`Bm%(g1*-CdhlTq!zm@ zfAp(%a_v1qcqF5$dWRqL-u?UNEI^Yoefo$v$bzk#FHbPLkqoZn$Y_VZx~s>L0)WtaIDcH5$v_UEOkcAf# z-EEueA)PUi?8F%X_<0~f7|Lte1h3jh&wxf0KFe`xD<=>jt*I%_Mp%X6%mSO%rvF63 zTcZ4{LCtE4OWZvla(M5P#@oPSgZ>PeP$dB~qpzOGOJ?$`=@AaHAyx9RdX#cp>tbO$ zZDeGu#R0M)mJgTSOkV_cx8QmCN@+V~{maeHBteIe3xdtOZAjIq`vZx&zXT{H9Qm4a zvX?3g>zuaxQH@udw~D4Rwg-4^{3!T|&M@Rqg;Y0KFw($ZG3$0bv3x%HfI9e!_?e5~vud=2m?GE$B`aZk&~Ov75AdtKCjJ{62N;Q`EsCk?Yfb1s;tsa$U)l zncw6Nf#Qx+N-a7)V2o%N+oug*t~i|(Fn$ERRbr(tx!e2pRJHX3s;Xt8rIPcD!~UUl zcH3Ls#-TzhySvIEZfwmL2fciA3h%INWRH)I?x9YApGzKn^njC#%c=X*A&Bqp_Xh+7 z(AE>baIqV-(S&i?RAddwcg`n1milZU@Kvm&tSsWfpH3zuY`c>tdYTx;%ht|r#{GPI ziAsFdNlvb7vhtPr635rhFEK3I-X@Edl3c{VH#tw|Je4BM?uwDJ5BGf(03MvHerGp^ z*@|LN@S<{~Nzl~AO3Q)P;phv>Ps^nvpQOqk_b-uHSXgW;1A#7ouBtlm!Vym6^6Pvu zk`XE_*Vnqz6EET1c^bdBjKhzZXy}u)^JBK9MZD~*BHxT8StqHAqGIq~`FDxF&W@9# zyE$-Uo%CQ}@Ku}s)d6v~Ez>a|mBZQqEVGc?JuooP|M;*6vpeAZLKmGMHih%=dA}!u zx<52YF3Nyu&9iF!>FAL2S6r^}HPdWMNJszyoC5xN zF;qlJIQTXZGwzDtAW|w#(3^WM(f8$J!deB);coFk;a%}CV}Xgd7;3o1!5k=cY4D<^ zXzs*C6sT+E1=T{-u2)+?fPcwLs6FHMu6AYNXtkJm`&3YAsK6Vf7}B_p7G8N` z2!i?n?rQzqyWWGVo&0WFZl1et(q5Pp{ zyvS4CD|FNh+}YkiTzK~*fJU*_4YT_CLGMAoGlYzHErxw^ywWa{g>-J`P*FaG0DX>} zBo1F6f)}FB7_#0^8=hzPxebPW$>UN+ZYwVg@aAswS-;JP`9NP!mIIowezhG7zoIo3! z)gz}|Fft6DRfVJPJhg6UA$riBgLGEKcddW2(4K zfAFafnA`$(vH&VbKi3-%zrkBMsYI|;ibvFzV9}$PoG{9#f`PxW5qzSwzk!hD&7FC0k;TsVX#;c%lbkH>!#Q##s1Ig}KsFJU z;IVpL#b8`( z#~?wl@JyNvUr$N`jAe*wbALRm0-{p_C`z*I5DecF6^_7)ctoC@gG0sKEBDov6had- zbjBmCAF5om>@IG6=07q|P)tbBP-E89^j#ICKDwLQF z88Qjma)kpgLt$P7oEuXo1k2XSvyQpz1%Q_K4M5z@7k!$u2sQlrK`_UAUJxM!B#(_G z=s<>s5*diU=8gC}0n;hkc~4Luj)0|`p!OMejSomFIM~9}c|#0&`hyb+j`C#BOUkRB z{Jt6mkQM2i@!vL=@b}GS#|g3asj;5jQ~e%TGtBy)^Dk=xYIKTLd?S4ljoyCMh0tP3 z>ivU;wu{E96nj|CVJ_;p8Mw-sIW+THH+u*N!td_u2ROV~#|4GU{d>w~^-77`PoafP1B#IRI?@?_9w+=X+?R_0Ah{`@&T(E;6ulF}v;Nz5&(6%WaIx;WIKe^{@Bc|kL z{FIeyD5FD3qNVgkpFAf*+qDC}2gW*Y`u@ak_%EjhaWa}-J6w%!=kC`r#kOri5DG!P ze~&ymzu`0f5rcA%s`3@K&Qny@T!{x5`;xX~_}MqwMG%CAv#!TJq9NpGmF@%KlKy4h z<(pca$!=mM8_r*r{ngJ|7DgZlv2A-_XNx^}wv6kDDW|NvUEya@X8ob(*R?mbZQ4lc zNI5iyFK*6sf>~;@{$}VF?$`gU`L0&SG<5X!RK=$z`&1yPUm)}9iB*j)oXf7f{ zP)M}xuR?ES{7>*Ltq)Cxi4!&%yPdUfL(~f=j;s!GO;dK%QA|0$j=jS>v^-eob2s)y zZh@2gSc`XLK!r6(1lU_i_Rc;qEE77FB!sQ4$ z+&&L2zW`d%6|MTbclhnZeXc4^pO^nv70UNzUfiHH@?d0F;Qe`Hbb$qb`dbTqI?B99doJ@d>Dp2s;M74gNpOV z;tLl>J~7e`g_W@09PndiTs(E9N?_SWG1mQSQ+ey_$RJvue|2xkZvDT$%q+TlS>S~# z7!fhB*ntN@Dc#$`u2Oda4T>jyStE9z^tmeQ{S_oy>Tb@9)(%H<9N*p1-|RCQB#HT# zEt@80BimiQj5A4-LdTEvJ<@3f%WLN|&G;dxA4vKJyIYt(URkv2vBs2ll1iN}JBdWY zOA80QcwcY38-dE&p!7$(87ASc_E#jwx|eOt+4Ku~rW&kQc?YCmlf#-_)!Dmj5Aj)% zAY6;;35hBj#{75zj%wy03?F|<7#aTPAAM9H@Gn`&WkxC`2qxAuL4j!hdMua-u&U)U zoAxj7B@4Ps)=V35##k$|^~&LaMSt?^>>Mqcx)WF5N9y+RHvfbh-XSm1(s)CE*~eRW zSNIKZa~AzCOJUa}$FE%+2*4U+y0vX=wzWEb`Hap74oh$55QNFWXe+6{)Sfo+ua6f26utbCJu>{A8$Nys z1Sd`b3^^~Hhe7`mo#~N{-3r__@Jue4k^--*>blCdJMW%Wvc|r$e*5(#DmeXCK9mMi zs;j1%~FbNCY<&u)0zkbvX{GaHli_-y(_suTaTej}yYM zHKGV|=(lE`jW<&J>*WKJ!r0&UQD=B`EnJdt$aKy6exhuB6c{qM4W~eFUD6=^hRcDh;BPr;eTCBacr+qljDxUM2xMWxq583-MoUtvo2{~{gtH;LdfU?K&FbF z=s~o%R#;U!aliQfvdruj$tPJk3EGSswnOhD{sw3#5}{e@APRro`Bw3`ohO!64Q5JS zl!q&F2Qv6Mo?L`TJmOa@J8bzfcEC+2%i#M-ULXbi{zeBre2)r|K-59M`Cr-SxLPl* z$or&T&BT{)`!4UyNM7+7rJNu1CC#FIe72XRS7?Mmj|Mk9cHo%KXfVt_o_-ND@j+?# z{;J3a@O*A%f)T!1apHQ?(|wRQB)ZCbW5sE7(L(pR#Fn-t!8 z2i*_3e{bQv2R+j_@*VR zDNMCrT25VRXuN&0=(F>xn~aUMkpE(BBMRq$$wrCCe(p;@%e2c+_*NALaFrkB&?qT# z&1^kj0%IEZreXC1V`yx~Y9`xD z7YAbz2Y$s$S^A#q(hh6V@;CeNuGWk^NSUj#8~3c4v!0%G+fL}o)ox*JOSp!Fv9oguht6wt`aMosp+u_Djdy5 zA#W>NKdTBPzUeouidA>dF^uL!wwU~z64q{axtFP2^b$t=l}S|GM4os1)A11nQG{GM zSM26$SKsX#qp=Oq(T=&b+P!vUzEizKFes^f{u+Ke)F6LTqm!=goc~(%WqV1fxuql7 zW_)?ty`#YAl3+_`tEY9|mSipXxEtmrA*eIeTXG@qMS5W|-xU z`VZe8@ycdqGl=Zevqcc~5rSW)@n=WCD?*0$LUrj@Lecr1^A&572r=U0w0VtPMhG*W z*I1@qu3^3l4C1~Q5#Z}6PLEZZS^(qP3--Uci5NRE4|60`d+jq`l2rsjLloJEDXb>wHZ}D*m=V>8P!cOG=z06@Vo(0>>s@Z-HzK8n$4Xk9sGQ2Y&S?m&Ewm>xsLEORNDCz--ehdDyLR-uV`Ot3=SuCtoa~YlS32Er}+I&CO6VEG>Adt4Iz?-Fjaw< zn|-%YG8{9CaA9xlX4pne8&XtCDpuHqj_xF2rck5M8YcTnE^(78hYjKaveIB6Y-Ef6 zPnh^Vu;XnXgv*SQczGDnNJXSEsym=`bLWiSn2cv*Mtc6@-EUy##v*SMsgJ1Z*9;o5 zmtoL-#V@dKDN&N3UyVgw&cxz91vB63IvEl%APnLHYw^F^{_w3Rhe;0u`oRVwgq?ts zeID~HMdH-p&VXXU-0i%0Ml`VledUK*I=a=r((|%rK37kYvyN5PqkFELgrvEW`r46k z-eXMYOLR2<9mx-uNVP^GNs!wRHB3raN$ZLIYpKb#fNwGm!GtiQc=KOd+INa@RbMmm zGUYT4nC-%7Is$}bzMI50r0A3w{QA_Os>tH|6kW_4P#~i|8W4+Jwx=jTYnK3C*yPT9 zz4IzYw{sy2g1U?+$xXEQo;TQr{f#HEsn2vzDl=X8RiO9ohMFT3RqQ*a`6@OWcNLS9 zU0)AZa|W)eQ(BtyRXu-w5R1G{$NN}v0|0T?%B)ba{@+O1-6|LM~ zTY%{AzT5r<_!ov3<;S&H+B+}En=7(+&I1sYi{w3o;`4(i$6wZ1a-&u=%Th_Hk9Iwt zrEwj?J&mNB=!o~N=yg+EnL9^-#N$mgIo&t_UG)Ds4E(QNNeH6u^e(F)8DO14?9$yZ zE<(Rja=WdhjCw&)BJQG9qJp>g{cOfa^WxTP`|9=&Vs=i%D>>kz` z?HQZ9d`U(XrN04JiZ`^vz|Y1?_g6KZ`btU#5+guIvEAEN zmE+8UOh7;+T8hE;NI3CE`JUlav&IV zmfZaBa(?vH!|CCK$8?LD#Um2zYwt zXqR)Jyg&JXz!CCr27fgTQeG~RTA7uQa-g@J*OVBv}R8xJK}dK@A_XsWUjb&=ai zNbq;@-lu`@(D`qJk8O3PK`&OvfgfWZ{#6Pv8+wqOpE#x8KuT>l{eRAZ3Ei&7_qH(PlN}ll~+Y{ z264|qCHOf62655oimJL3a&a6|Hdn9dF~*u~lFT<<_XnFw$22R8N%Akv9wmE(c2Bgc z|8M}zjgl=B1H0}>bLM$yeVRhIkacTrSKv*1dT{p%IZ^SNROIYlx%xvafksCk=-Br* zS;A-IhT|pv!U+LOv1#4F#p|S~UvtSkITcqaj=F1c9>mqlD^8^%dI{_tIZ-jpMMW7@ z4;n}4XVq9&l-j+zBU3{98eV@z5x}>vH__Yr0Q^KY;-~6?dorIL4JlI9OeZCKTJ%jaUrz&G!g4cW5xaqO?d6y*g}+s z@|3m6(55~f7VkL-R|k2OrA)z%g~%&>=i0?(Wb@%E-xaRJFa3!qmv3f7BHPY2(>v(= zS$vncT$Kiz;<}fivGffkZ<$z9o6im24B?)4Li+J7B4>JySJG?sx@{q%Dt=(@MigcmU zg)!nO37ZDQ4>7Cdj)kOK;NnbXFP-NRKKwVvFcDQ>qW8j_(@3@_FxT&75+771(ggaa znz#`Y9?8rY7Y4Z2F<5-^2E(*3eSK6z@kvlkY`D1uqFoQlh3Is!sW5@1J6Z8IC=`XU zny?W{9d^hHOAGXsyTQRM1?63F0x4bRCq$H2FT#YVT=BQBMkt#T2{*FL@GC{tr&-E1 zYcNnc)K2{*7}Y6WlLpR%?6*tcw8aO0H|SqY1$&?4SWH2b*H6OkyJlo%t|ylDGVAsD zZ|+m*T9I0smIPg)Jt%}|ERf%J0gPECT}ih0Mt*H$gpT|KMF1;4?$rF^&$>9f}% zw9-5`=Ut_)kqeKGD4h;=T`qKPoLbw|)VUj_$yS6}V_z@VyMHlsxQ72n+QNI>W^1E7 zrbMLRBgF$|4Z;;r>ETVpT8X) zA(jy0ZIU8x@mxY082R13+Y72rH`;&LW;=w6!SNZdS9*>2BfJ-k#QUpv;OozLB*3l# zncNeW@26<*B{6#}PG5dpAdh<&;W{V8PmYP|=z*(Ck6*?jodznY;tAdFl5P&9 zuFl8TSrKI?njkk=?WYrR#X(7=}6w1nUFZZMk59^Qap8PpGPKpCQJGW`fCUPXq z4L!4~DQdo(>RfsGE?B;@rxn}|{a&2#?4OR&)KN5^50s|AiT+qX``~sQRkdbZr$sW? zrjo$A{En__&joRx#Fdqd$1YA>fk&1Z7&h}B@sbF3D_=C?3AX)_iL%6HX=5a^@Rur z$Tek~BE7`HM6a!f)4`qtFg-zw-A_o9@rsYdHac64bK$6f5oJU+qg%Aq04!4tW08R9 zk=@*J7;6%Uc8C{SDvs1`65QS$)wPV!Wvg;+?+#4r+GB8x6J0P7t{oSy3X+F#s03i_ zzQS@`RvGPj=n#CaiOjC)xFV}$IBF?>uBp9jUz%9hPhE__-`uWna=g(o6F;*aUNv>< zp2Fc7Y~*}t3yzPOxa3eUY<*$lkz6g7Y<7Wg2y^WLv{0c>J;!7;hh^mhQ5SJ5G6u#B&u7m2 z_qDgJlrUBD7x{wXL4a}RrQ(P|hDt|lAlXDDueNfW!6>Qb)j%d5#oe0BY^dS=I8(2R z6=-;iSzugW;kXF$IM%MYEIg=04W%@|bgxs8UPNLC?0}W$g*I}CMr94QUZo+G19$5C z8TG;-NGRU3ml23{>Cp;-W_Eq1*-w0>jDTDeJtf%O`l6thPzt#@Y;=1rc^yS-9O8vst`M$-1E1TA0n z9FvyZteUQ=ZxOH83VmX#mU({Z4Z}Y2oqZ0i3BgsxY|7R3?wZ0ss^rj-1pDUxgo5Sw z$r?7a4MdD{Wge*|1a1NL9wxw?Z89g$SxAES?I>|UTV%%QN-x^_;^Rl-vIKXn3F_@r zLSDvw0sAM3qx;+PV|fC5w(j_zXZ;J&+BME`6kY1t58R6Tsq@va8I%1A<%3$Wk5@Io z^52TXz?zj ze-_zb$i0+ml0pa@^(diEi|>+Z(aC)~AwW1+`hE;*b z@fjcf;ohX9m5>#!O@<`=1W!*5szbFqBQgm!T!W@T|E&7Xl(U|K?dgr#$ODIzpDcvfQva(*uHe`i;icthS>}>Lx z!XL3*qI$zO4v*6n7!mKfa=9}5d*HEV?E|r0O|SCQ`j)C)^p+zB9Cy~dc9)+d7#hkWrzQ2VV!yP>fOn_&}1 zTkLDna&m2|=AAp{#^mEd&RsXEhASc(kb`Q221Yj9p-ZUoiA?6rUD^?gD4~-tj91eZ z&N(K;WZmrQsc#ZI?xqfl_aY^GI23}HjBo6Fd-ibJt;`U2ds?ZqwV{fA+LOk0)s8b; z{l=cTS*>>D414dn*zk{TjneLJZUwI#F?z09XLK(*E*NNWZ1-kR$wID(oH-lg=JjW^ z*9&iXO}br@v*S=TGEdMIHSnwm^*_{Q5V897y3vV&b+p#fJ)HTKZ=_tCseF%ZP|3S= z$9-SmgHodB3HYyV9s8(`YdzK-VW+bMG{be- zb*QG`GFf(d>)3s*!0=fesB5c-(q){_A9gnUG@n0}5>qn;qRY79v^WVejO^ZrK+kbw zX}wsD(vXkb2J|JGgSjCCIZ!jgqBM1sb7{{-<><=1khY<8IyfJFN-eqFbN&yV%oY3> zs;>7JK?D~DT;9>|eph0U$V=v0%(0w2j2uUMM;_oGbC=pwWc5cb+SH%B+k1b)PD;6_ z!@U(db%-~rH;YMsw39g@PIlgPurLxaExZ2Cye98KrhCq-6_L z^nBMmdLop*l3-sv3nJre5V%z>{iFyKJjX^?y{T365g`I*ohBW z|IR&77;4WL(WXuZ75~zcCbJm))^u zZ$BR((eO%FRM~SIJ?#Dt3Cl@TVt}fIe!R3Nbq77yxVs#2!FHju{CG1k%6T0xztn%O zzROUbxCrSJ+S%oTNp>YNtkf)2yl#s~cH0#ezdn#e5bMAV-jJ$V2ODIV(5#-S^t~qP z7>HQj=$Z^*94EIR3JP38TM>J6`{$^+=3-~qXaSHD*v6e;xxjw=RAR4GTMhSU^CQUB z%1q3q_bvC*OurP}X81f#abAaIONd^uCMrxuIcYX&4wiH&mtD96JNDvYHP!d?9|IXT z?A$^0@2S2lW(%Ucnj2_nc)AX!9At(fHYv758*A~+KbK5SLbVQ~@kMZTS9`JFj?;cJ zzrjmw?AC>ZiHX|@ly8}Mj)P-H?oWi?c^E>3UL!x_Wcd`9D@k3oahNvwjE+aADc#Ll zcR~8PVl;B9VseaQ)%TQf2CSR4%Ti!5q6(5BB8Xr~AN$@&KJoaXr5!YxqH=T*ALOxO zJ$e&q-RdXNOIfT0iR-u&Ax)Hrd#r z(x4Yw?9H{}WjK!X(txz{kGH1|!?DdlXE@;|6BW%)37Qe1BG`F`y9XTONQ7yt?u0RT>+Ak(E1G7C+ zHafP_r%T0{-b^1)t75cCxh*eKL|79#On4HMA#%aUOx*cfSXHh=CugCtt8R7cZJ|TM z4MHgu>~BOtDY9m~)r0jsy9|NA{?RoKa$@tEGev~C)289#!I8bAK%&l1PKtfiP#ATp zdF|V*9lVXBFF}oPy-j0rO!W^B6VQ95@rJg*;ig1;Qh%M=a11(t@e{Psh;TPV9x_yU zWuyxv#0Zt@Rj6iT?^&lj=Ho%xatX0(%ICGJ(WGK+COE~8WJbI}2E3+vtmt0c>SC{U zg_G&b`)vLnwR4Z-EpX#FXUH4kB|8t#;bHlUVGonl{4>hYf1HG}*mIC}Ggkma<0 zB}l^VPs}2X+bBj}EDyxO%hJghu1Gwp?^TOvShUt#r*NKbZVMizwxCkia1lxki+8-S z0Zazxl?&HYthBrpY{01(HR-G*KbcSz8v8V zntX;E&e5vb`{F~-`>n%2Z)LcyBmXh+QOysi_a{?}me)I9WJV)`rsG-qhM&uHGad>Ma(=ru2WhsR^F>Jf)$ zaTiendfs1~C)hHxQPump>W#$}j2b-|UrU+qQ`*dF?V!*2SE6Lvk=)n7Wto3M1K!|@ zQ2qeRQzBLFgtm!p!OU0e1~&JMs5LfeyplWDKqMfsRDMVCP00=ilu^plb!u6z{NVK^ zAAfm>W6W&eoAUKS&c&6*@^r-gat_0ngAalP{jnhGzMb^hc>E}~SdU-%swC#-JA+DWd3XKWaj1SHB|8#hiygI&|>Zgv;J)_^$H-g1T@M+HBO*1pSB& zrI4yT3V}kVUbfrI7M@lfq+&!gwO0G5MwPsbtK}BVN~-|LJQ`{1TveqR%$Kqx{%rZG->+`(X8r>*mO`J_v_HyK%Zw!mz( zEUKL#_l>Z5@PcJEyX(^LYb8g4v=lP{pA$+S{H`GH2gi z@uGQZ9j{r+;JCulFwrCS;WFe3H@P3d7b)wL1nzqk(}g(i(sJsgjW}7oq&f3)0h?>8 zIKExl8dl&tAufSywdAvl&%~vAA+C5PTwePH1;)HY)_#d?@yq;v8c_sGn; z_jLfV;MFYD=hylA|kzn21oSU`^UJKFE49HI@!s^zuO_W zZs}PK12o6p!n9dQqL9T@R%4sI1gE)$0x)-vi<=lL(zoQ0GlQP@i;0kE-a!CU_Ka-> zkd7r;vY|gZRbBI$n5TltNv&G-POI1tAs^3|SG#g*G(WsIhyldaa~l=q)%eCH$W7Bd zuFaRjAEEmT5leh4I#-pAKThyq&3s=3w}EgPqh`|I5c6zb|H=$k+v2)6)IXh)z|0&i zuA(I`!F7VfV6bp^y?h^W6GtYn36AXZh?j#40BfR08@HOB3_YF!a?5HYm(4>bS~S<|bYNKyz;YWe^tH7RfeG zv~rKsW%Wa!2G!^Szq}8ZX&eNNb$b@1SSULbz#-kPGO<(|>7Us6Lk)U5sz%OeW?8gL# z@o2_Ugfk2UEvwpi3A-a(m~_<=EvcWf5D|r1s{6mK!#N)IGxW zPs$bXJzA5$uOk_3@zLqSd4|+v_lvPUz#~YtVMtBO*2+0XD3MK+RTyl+=U%1go{L`n zVmE9g`zxGKn#s5}?nee$3i=s^M8pR@FZCX@ti-vognW6n(e|-(DLIFqBkg&zk|Sns zs_kjcnN>AZ)qbrR*+CCeTsD<(4HMT9M5bTI*p+t)9zPQF|D8X6)`wj2EQ!B8xus<* z&iVa&x*#d+Abw5*#Fk)INSLDN7ZfH|vV5f4G{(YDhRzVN-dsR29Iq46>HRMEvz+tnY1%wG?gOm% z*9C*c3?%Imwgc65Dpmo(H^qLR+|&x4t=lm;{FDx;%R8{S88^ogE`)6+`D8)wcV$P` z$*KMv?~aq4Ll*^@?mr;tg~zRl(C-u@HuFJ7WO6*)Q${1?DKjwZBgdmrsQnJ6GV>M3 zX_5L_7nA{M1!`Yl^AyV&5^I<5-k&JH3>LXv=#$C_`+mg1m73PD5(U0Jgk$^G zZM@B-PG2V<-n2_8b`{su*#uc!FDZDJK2~5$ohQ^Eawjxeg>UgxvrBe(II~gIS{>{B z%r>AM3~{Htdp+DxtzRQ0jArxMA8OruFH_w$D}O?Jj6<5p*vysylp_;+%QKQB5z zDNe5C8lJ=N_ZlQ_RWerir$WUYZlN_MfmZXq?i`W;^QZ1y!mJleId1oS8RuD+pWR72hTJ^}bWcO9jb z{o0D`Qx{3{fr#u+K2sKQ?RJ=;-sgC=#jtl8Bqb&qMs4bg}ng_rn0B%4!Z3{jL>i$Q`)T zZ&uYQca`KgxQ^^k%9ok^P4|zTa6`CARK7%r0&?W&u{-1mIDa{@bD)#|;DnEe=>ruly0;mNXTbv`q$0np4A5zk ze3M8{W-d+7J{rz{J*0Z{(7Pis-TyuWs+uyLIZk<#{_c(7pId@oILsU6KZ474{@u%H_x-H`;X`$dldmOd3s(_jeZXp;Z8nf|-{H`4Rf1>{ZR#OX#|P|@Q%MY^CPAoh!#KFno0a-E^Bg6a+9WJfs{qB1y zOfQFA=3(~Ps%!-&@(F;I+zasO41&Kh+#p*YUZT8814FO(>Gq%mE24S)HD2qmfU|gJ zBhXwAL7h@J3=l=&fRUs~cL{idJ)c$J%eL4>Hmr|?osLe*t^#S;? zO|JlLX|h-=C%^(uVnx83`GkG zBgV3PG}%cv&5Ykufj--JDiD z{USe5$AR_q2N3NMri!)u3}9HQSJ($9=wzV z(3U$Aq8{UiA=Lp{sHiLn?T=Cnsh>tJ_MubpFc`N{#s+=VE_UxOTl?6%C~iI6*D%N@9o#Y6uKogqVZ@%gQl#N!=$$LWb$7J^ zJHA9x(*Y_l5G%?+Ni3tzR&wP%Svm!3w9NQ%T@;~tDBY!^j5W1zKc{u4>PkEuUsw9Y zj6vJb;T~XzB=Q8Ye^S7`f)+08!$u>c`X6_)+Ob=KkBlxAX=4k=R_wzD9FFA5YWyH9-Nts`Tu@CS!$aetz?iuEr<1);+GMd1`O)l(K^ymUs zQ51ec;*O7A*hE;_i<32fBt-P`xXF}y4k@dm!tF;eqyeVTj7Q@y*aK)RkN+n+0ciKb zM8k=`SrF>#$#=%9mvUAdUe#6-TU5gbqSC?AvM_t3JNV$#VtwxhSOr8NF%ZB95lmH9 zfL1hDjm_=m=qC|$8&0P;6nQ;)YRAh4sAM*4Fw#f50Y-`pchTFHmDZb;L+IJ!SF!iB zySum6-X+!bqP@DXJx6gofofBKI-3$)HC6&b|NOa{z>cVHhxp2vPk3CRlq1s5NsEeM zZ#3GUFEAdgy<=jrS^F~k#ULy~okx^^am5}Q6l#K`+u|~IZz9=m>&~8RJ&*+iWhoif z>&>r<%SIDS^sh4P4P}X9#F_NEe@dEYW($UgKhu-RITv~s(>%JtU4jQOyVmmw`w&I| zqq~TR^g3J0ZSO{@_6!Bj6mV^YN!*;X6t{5%oGKM;*ES7uz$wterqByCvB$whp<6-h zFj{#2VqQrx92W(`TL+k+_KK=dB$-AJ$7bLaL4hv?d4b4q*Mp!!ro*Fa506n z@z5dVm6QN&;~Jna`7jw*C|#Anh^#RCn{V8@Bomrb7eO#d1KKDM*Tb^D=l4?|X;3n@ z34<6PY`0@^NnnGrKFjAlh{3mhw&*2+b0L(SE?XYMFpKm!J`iz}^n` z5zB2tmRFAwQior#!LS?T$_c>Q$!or~vAZ;m4y@!=De2Zs0l|+ifMs=+Ykdl}sW5)Y>&1SqY7ziG8qlnL*54Z##bo>yXajMZ`y`{uKfEYL>A*1=6A+r}) zyFfv9;N8^mH2ta2?n-KV%HSMjCQ!c|1e1%990r#%^IcrY9n^3sg;=TuR7ZAFjysuY zZ}~q$kqHavJmqsloLgcm5VLRSn7T;2qK~?5)uO;tYMZXJ_SCWCnX&Pb5RZAv{fAv@ zK~)_+EZontW7j)Lg!;jwvR#`d*1&H%!5;RtW%AnLbZXZ^i0oJ9(@0sXqzU*JRA z+|})a$Qym&(Pxlqx|e4KcZeIs_g;W%@aXIgE%veRRo^ZA0cuS)otUm zav;tIb^BVK-Ww&?-Zhnr%Z`*QyjaV2EhAU{XmUaQG}S!~A0?qp_VLuNT%W$Q6w2{o zS4Ddzova9xGM7;^a5nv7e@PBPqx2zu-N1WWW!L9u^IFDsjoS&Ut&Y8#IA5QGBA_F{ z20y5090q2@Ah&;e4VtPy{z{)tDyNRHAW}xm$B^)cuF~kDy{6xKvl1`xY&%=vH&rK= zmJ0#9)%@7Ul3o5q#_h|G3*U8~ax>x9Nn_*-L$c7cZt2MsZ?+>=b}Q7r&NaCzR7Q(d z>8dz;n)oDoEXy7!93`KSlKz%tW&ur&5)u~>Z2ywE$=#_k!5wddBR|NAqaHB1;w~`K zQ^qFESyD%v2zOx|dru|iF$4X~{wuuI;q*oG-b0mkLPzr$=5s)AU@UdEmJS!VZn~Ar z&0g)DbJ}3nU@}>=$pKf<+FmKwvN#vIfH4I4qOM!zuFm(V&j1D=7xCj@`*a|ge0|_} z`R4GajG7gN?>V~;%l$qlOaLxBGuL<2Xnv;{Z&s~*#K`RVh`oI5X|jLDz&DcP>KFZ1Jz4F1{pRdvOyy?RCySlL0K_Bd!_ z2~4Tip+l*$dK$9_81tb0-(@Gq9y;SUdg@r~TpoDp#fnDi+PObK{3DV1A#DAa>uT+^ zbs3_XJ9ZUD<9wQKt?{ADnF+=U;>@MO+VbOr@2z$*MK;GBW+#^ynVV3Q}!4O4s&Xv;+i|UU3%ew%ZBK8W{q2Jc-g3W;zv@oK)IUjab#*P$(#?trf zy$0b_r>o%VDXL*vASFQJ>+T&gGc*ym`G-7(vu3mPVjSMmXSZ{Tk1D{-TzjmjYuEVv zoP#nI;=Bg;*%*33rP=zXakf5J8WZ-HR(9a8C&4e8lIl ziUfypxE!<(grdJPO<=gNwb?#(G-{Vc02%Sldtw*T4pKJzR7v#cC@AQv6J`K$llWz9 zv>N_TjK3_E4(uX?iSz}mPQDc>8GD@H_ZXGxcq%)l= zzIbndv7-UkaG>OgsIgfBq{aHP)l4Rl4IoBkWe}Ezp|``_U3Jx;{RqWYtZIk1z`lDs zER-9i9XJ`2$9nR}?Er-9lWw`ts_Mm>1H`Eq%|_$Ki`HO2egIWbwC9fE`Jn|$ z$d)R8nxUyLr*Cy4EzY@QDvDjBZx%KP-{CZONO4$I&t_VQh1ET>`{c%fo#r*s?0Euc zd5{3V$*6K2!QoDg8+R>6RCmCJL6ido0PZ-T8etS_FP0em3uY(Bn4+buGM)lY+L7c2 zfKMR19H}PoVQbSY6V&VrIab~Y5>wRvlylO@d)wcDXU925I79Wapoe;@JN?brlP+w_(4-xW|A+60|6iG2I#Jd*Y&t~@eGbc=m8SsA03T7AS2(?b=~EE7<|hi zPWUsOg9kt)O0i>9ZGlOeV`MIz=_ve5V>O^~7wiE=ypoV*Y4cax2l2@znyJ!|bb1vCmR=8}-mJ2g~5T^RKB0ejA4m@}7f zd?zI~G_G@y{IP`N{C5Y;fswtG2|c3a{WxYHgmV>rD?kLP`MU1Qm*fD<)`t9ohqJpP z2NFY@I;xoZR`-^Vp02$*X2-=Zr@3fz%nJw;hx525?K~YhIXchUp7jSMZ!OmVI!S;4 z3a>0Dz4@G4P)!wl?@H=o)Z?&NayRW77tQTX#Vw4T!-Mj(vBgK=Na?duRs&F+<$(QY znC8>P5p?Bc$~ukpy0wGy;F7po__!0T_Vy+*fl)|7Y)5_M5%771;gk77JNsr}5iPKm za%Qy?Wzc2W!f{qk_$ZvJPVC^Q=!crzuqHbL{z_pAr&>D6kL1iF&s0B$p;d*7;5a%b z$(RzK?%us}=1xvy!0FF$enNfkQGkD{xfRq*FL;Ej#(MUY__k&OKxc$GRu6FoPPO%D zY(9JaBlpeNv+8bMdb;lFlZ>d4E#c8%(MlI2dSdlc0+6pSio=@<_k6Oe-j4kO5yq&% zO*FePRyaV*?U5Ccct+&PLKaAWLYvGl05~mG8Uvt*=Z@PtF=W>o{^7*0d7zreAl6=F zq}t4J-nmbiL)~gd&Yu7=*~3#n{<#0RApYy691_yg|5<&c=LEl0Z5|PSz(z!zJW0xv z5CgpGi7!&8KvlFa3D{n^c&OP#O&4~pW9$Se&p#ji@VLxS^JeFxK}O_DE)B;MbLXzP zds)!E^~bc5n38(t0E3z-pgBKdf_;B;p}GB8$nA+^*A)H%JKwo7#@Gd9RhbkEKcQ== ziWtSUN>6aLyo}cw6J7u*PPbwO=Wf63f8&k?#hvH;$OHbsDcGG{@s570+c4_b?p)A{l9?i+i36Hwq_8gz6r#1!#J9g#NRv^-)<^v=hh6Ufm@d@u>9@^ zF{UlLHHf(C@X#QB-$BXX-+f{j6|7IZ0-w$F0&jWtySIQkH_l78n}s|pSVu2iQtscN z$3jDk-Y@fDz2h0K>-h5DIB|r+{|~)Tvm^&Gk^k;rM{3**7=|;FzgZMHtERH5 zhFL_eimt%(Qgj#I{-*CxrrzNB9g6)&$;d{|@~wNKk4Q~rNq@J1;0h2EdWpv}umS@> zl^-3@^e>2gFoM59d0kadG=)ba5){b%_JdGV@I({*29xRM9L;@=%EX&sF#m^IkyC<1 zjf#R&|B~)9zi5+|U`aldb!$!vo`uDR0 zD<%;Y{Np-#pMgH%qU-Dyk5%b3wTMu)TD}V_{HikZaUx&>0!HA>Rp%uPAG&(}9AVn$ zedD~Afr)wnJG0YUO&I8(uXWEzdSv!oXgHr!bIjZ0=n~zZ#fuI^R`Ct6WuC-)IutHY zY(LvSOw|HcO3~kt3gF+9Jx<1*BlP6Txx-#6E!MME`_Jwa$^7hQ@E^5OR0C~^qJ4R1 z=4zyl-=aNwqA#B_@?=DxQ6v5L|CC3%)1j8D=2;z-f?tz(GME*Ens^^**Q?Olshi~b zY2?J+u#1|{23~LDJkPhF_Nw00@G8cj*j^FNmy{_A0bdAA#YBy%p}`l zxQ^JDP8wzG?D7>so1Eq<7qAd+Br*RXb?F!t?OS@^U)HjeR0-l**$QeZ(^lq$U-cT$ z)h#CF<1-t+7cJ$~`doA0W?82OWD1!0;0I4|-*c_$^NZcEZgEQPr+38npEPhj=3^+R*or=%>A^FaFvR{k-`Dw@T5a1z@CXIrzVA|^^2 zo{uMgLGu26{GPfB8lWP(juv0>1N%<1`p!F0(4HaW;Hr<&hcuPxCN(|_*1{SF`j)<4DZ_vg5tUtj#=#oCDsd(waO^v1JEn%5ry zNpd1NI319k$3N#pydjr#{&T1iWHXKTjvkuhJUTP)&oW#yGpe*zvIo;|QH6W_Nhmj6 zdWnb@axgK#JS9!CY{UP!rSs&i64>rPbr&>Y*^MvA2AG1u=@uvFIOU6JMgDrZe;@AT zX8@3sJYYL2w3WnYojX$dhj842YJyY$&Y8;ANH5}#JvpLcuU!NMYo(vw`zmGT09kQ|de^r}hAQ-MXr(>=Pn;K@dQ48I2rz@Yu7;@kn>Y^l{9E zoZzRRx)eCh_E?Jbv%$Jgs`G=rZ_%QH5uinvsR01Fa8MAPRs5uKU!;ezu$Re6bb017 zZ+`z}a|ybY;z+>T9o-iFU#<4AoRsbdEKY8VN6mQ2?VjlwUVi}p9UYQbQQ%GOICP9K zXKLfXfR%w2`DtufN%l+j^^b}iN&;l*Z?Pe(ikuA1Z@e?!))K@Do8A#^BM$bPky z=cv(uf=h7FefN@{68pHU6 z()5ELy$iK6BLTq0OhC*IfP`px+ET<_u2MHXk3VUg?srXA8C5WTUSpDU7aq7a&s)4c zdXGCU(GsMdC%Tp>0nR)a?d`1dAJ%N4%zGhiY}zt#;~rkP^+PhVm0gAN$(EyP>XwI~ zK$-iNu79mJ|EpZ-LEFd8$rPc^Uf+OH4HWxx>bVm3L%i1xtEv_|8`o!&2ZNHC-vrsc z`34pm894NjnH*Ch3x_h-*`&Sc%q>Dq!bAh-4Rq+9L0R=|V)sM6?ekXJ63~urKT-wof@9_0kjV|c%w;C0wE$pISd4J~%hix0*)F$DsSXguUyirbryOVO z0&tOIT*dyZE}#-X!V6Mh9JgXjj7BFlcc#e090%2`FKPf(K-jCvIka6qWIu$1A%*z z7b(Dh3C8Cu@T zJSymdl;E0SPm5MprF;Dyd+TWgtzt{?Q_-egK{@O2x1jVqWe(K5yIQ-TK=lfMMHwQW zHmE<2QmIBRxUcVq0s5n`9m zP(?Y(nADwT(`=posI=R$^pjfxPn=Ln*WJwcL^_A-%@kF~LlY!33WD|uYl`xY$A2KDRd|FFqs_K+~IV+pxS6}z&;JQuRU0z=@L~7un=yc z{FfP+>k~pLD*??jFs^&r{lib^!I3MVHMzv6ff=)0-AEreYH#g?&Gp0m_*f8jGn z3m~VCMgFfPv9wC%vrve&lu&P)d|1y4#B07Ns??RYx5%FYz)c~tV+GV-C;1!|4K1a9 zBGV}As9uO?Js)fAK6^{_Sv3Ik-i}8!T%a9Iz#d%q>%-4+7W$NKVy9p%Ul`pu38NwW z{#>4)k0dAmZHB=4pbIzAl@rjr1$z~g6x!B`bheEx(vvbq zHXZ_vH^JHTSad+Xd)Il_@E+{~4{)#v{&~E_*k!je^hUy|w2H)JR-nEu@bOU_tIq;u zi>VO?FVs{4Uyj@(j#hw6j;VvWBc@cO1-1?xIp&cn+lL3|gFNdwf;eyq|n7KCJgv79`k|gCh+NWtzwbxn1_; z|<^Hysb!Bek3^c}VFuFHB3Mmnp#{7V7 z`hk+hE7LqPvzB!DbVgSa5gmg-EN^VoY@CK_mKrsG)W}Yj5FlQgNUe5RQLH(bazi>T z$yrS94kY7@3*F|Or|rB!Ym$hOB_^huIlZ`IuN>Q z$}fvL>R`8<;*ng~vaKmM)L@Pta!#8#-(0fNvMOHc(7QxD$0N2gjYce1ln@V!Xb|ho zoy29_#64pB!Rn2h0w-E-Ty4R;yUnGpNUm8lJr-OM$FjaW$4T7ZjUdkD@er`e_XTx6 z^IOW4L`!WH*T)~!1~EK#L(CUIuTD7g_61M#xOdS6BKI*8_%h-?e$s*1Jh9J2+$@R~ z>Q!Xgtx82HZpyB21`&fqdEW24deh~vJ^m)KzuuL^bekY1abO<5e;DLG!v(I1`nGBT z-=Avfx>Gr8v`ed3xxaA;iI;Wy@sVCzDCEBv<bSE9A z(GyXJ>B9rxuQ3hn4^}d;b0ZJXFfTvWv2^Af!i^~!@wG4L!%nCivTKug;KOrJzbZkb zoUq-SJ#F+wqW)tS!upVJ>|n>A=CJ3`UPiC#kOqzT5I}eA)dh+l_R`_h87bbGBPx^k zaT-Ky4t%rdqyAbXkzkBpDgCxMxxcM#{7u|tXAI}QkWh&BL2Ro)5j$li42M6g(_6Nq zg^>FNQ1?v&le`e&n@7!dt;B9Tj~?7*_c~_D4Y}R3H%SahJsi}IsJ3cBA8swd_PQaP zPQ&M_4`-5@_I8a?$OZgi{d}5TSIuk*0(~>^zBpLAh|Qowe)#4#miNQKx*bm?CW7f8 zbd#r@sFcrc+wu(Jn&Q2zm4VB+kfVq1v_lee$kB?ohsrNBS4_CC6wy3~ucTJv=Raog ziR8p0HV&F_Nn%|!M4xOpwvC3O5<8H_B9sk1Xi=s_uI=FtzkxAYn=gT0J*?Z>-dwG4 zI>gm%c?Ie0Hp60xjyn7LadYuY6!g?{+OqxQ|jCAW3un1aNr%bST7lDplqGpPpQNCL}3t32ss`;6X;*ztSF`8qhB2A3U2cTDW;Y1?vjbJ@TDYZ)g} zWwSLjGtsZ|^8+Kv_4FD+Fz%I;1DeGgdDljfLOSkg%)!d{f_sa!AgB+RDZkL8uc@15 z_Xo$NF;_Uw^>{>3z;1&lhWlrurRTNC-!OU|7uL-XxRIszP6tZzJoee@>px$wmBzRD zrk{PA;-0_>*GpjIWeo&J3bj4Zma9@UO7Y%OB#>(yP=w6&1quc#M4%Df0K%1#}>& z5%#WQr&>P0nEW{!3L(^YlqlaAU0RO*YdGk#nT@&il*m4@XwZ`Es7KA+7n{lMJ8I^| zGFQ?OulBy+YVZ>exqfyCMvkQ3y zAI6@CFRGHJ6%C@jg_QlSBzSKrkVjgPve2xI#v^nq$-j=?sdSNRApNUj9fQHOm3~Q3 zA>~*lZLNui5KQjm16p$o*D+WCg3S`tCtDrNJ^#bAl)+GtC9w_*aC3>O^ zW;1B!>CYo@V&lwdTjf_353YS15q6$QqC*v&6L=NAe#828QRpVt^H(#*E@J$u<$XxT z`VS#Nua+9YD>wS6Xr7&P78~YH5Yc}5R|{6oL{R*3^fx+Q6-atms(0na))`p%AEg-< zG~|(7d4J6j&hzM9YMuVx!0&R*r7>gt5XDZ7$1_8K+q2U%{@dldH6}wyRtiCQGN0D# z@22qx3~d}8uN{W!b;Wm=p5WmNzdDigqf+78xALK1ldnrFkR&y1CObbU@yCGF{Ijdv z)qPYgH?Ou-@m6ZGp+ga`TS4FxyB8$~`jX!tOpoq*g+Nh0DLaU~_bcFVYr!y+h|V4G zN4}wZmS7?9N`1yf;z0-ZSc!z)b&6kWf>Fe7JwL;T0^` zm8^qQuj8rV ztBYx1wsk-N5^=85)!oj9=Yeweeb+l^neXjhzdGI(>pZU+p>ony@aicVoKb?e>MbPs zQ7qww-${kiz(^MAIhEt@;X2{c6du3kQ^yPJjI%5c;=M0D1Bozb_MUl*1%7M4>|74N)U%yizomZWLw8G;xa&mn)12k$3Lh)8#&Xxc`Bl_FDVw z?^)l+XMfITpR?E5Nij_csd$-?eCpDFis@>j2v<6pNhOp)(p?=OsQTXqx8!t8~q5yi<3n3S_Ld+pb`KTsd2C0(}+Cza}e|e%=KSLaoY_ z*$3i6)W#ReT!dDvahGtqv!#IxpDhS8QW5cDAHs$Amf}t54=mn?g z!CzjsD~&P%lQz3shH1R4r$4PPdT;A79o)J}yWLh?G+xv~Eys8edN6f2JEKjs5=As# z;fp-Z0%^IYXUAuR_p|^uxj6XxW6Z07(tCk+br~VEB#)nlqTJbFVoV>ek1A_d^SZQ8 z4DxX7@u>=OX$#e`RkW3|bFR>O=8^LPH^YI_@ec}gpn=2x>65bU>V>Mwa@4_`1QPJv z0!1xn1po$jvV^9U=EJvz(g4zJv2dTw;*Gx3_+pL#wia=;I7R-CMn=|RI4dL-u7uz2 z^!mo`aBKp{&WoxWuCB3JNrO1MIvbV)jdzO;h3IPczlANd$5m;Z^-^96l6R@SzJ=w$ z)d^*S1ARr+cGICFa!_lDfRqMIekZ9R>*BO_T~nAuD`>g#BXi6YQ6c2i8+t%+)Xf`y z8IloR@>iGy<4XnhGD2wTu|osjMIJ#VyEDMcmH2hOP$oT_OJLyT`f80g+i`Ug8i&_X zXAkd&Ou+N%t+8q|iUT*8DHeRcU_|mZYL`PSwFV15zWod{=|ewHuyM6P@_rwIJEQCg z10tO5(2zmIKB?67N_K0TISj6unpuy>`LozqN^hKZ#T6${f-Hy8;JGIhDC>asVvLaA zInXvTVc;BtCbN1ro_do27VcAwQVb$a-bK6-1p+GzPL--Q#d?};lGGwsukm#r9;m*+ z_rnNZ@I8q=r@~;3?HQ*Pvoh^|SQQOxl+QLl1}1|8Eu~q8D3fDu0h>pe%Klhftpg4Y z6wps{&m@PZ3+%Cxx@RYKk?V!U@1D($CHQA;LKSUdqr3baFyM27V+=(AT{%$su0fOe z9D{so?^6f)PwYInf%P2YS~Nk@`$V2Q@ObV8)H?j5H|>xmM~$}^HaR+(axKKin;#j2 zb!{Ex$q89UGP3j8$kx&0vql}8VXknt$&;d9sEVtFf>TyEb_h|$8ZvrVo*rSJs!xRb z0p5eo2UdTag+3XYE8=K}B5w6g17}k3GDTHk-!PY%g4NMWlz?7d6@Bp&ZlEjD4IBAX z1f5wWR}|MnOK5qvZOh%RNHrhrSNj{!THaWe3SOV^8HyE@8T%tQjZuyjnRUS%ag=D2*xpe>foO7ZmBo@Mi>UuVEf6}a0=N?XJ`O#6Q|`q ztFHmP;uzGRMzX4@g={2G53(z<~e(0t9yt5C|4LxI2MBfM6jw!QtQ>+}$O?-Q9va1Ph1Y?gw}K zHnID;-}jlRnm<$XW2&Hv+OpQ_UcI`nzPdO5^0HzmNO(y1?%hL?5EoXwckf{z@LPcR z0Jw6XTaI_{p8Gus;n&IzTHDF~;HN50H*UA0tAn-;gSJT>wdgoc^_7+F(C>?&ix55) zQ^0*I3@3~(ETAAf{RFXIN82ErP_UJ#D(uM#gMb%F8comvxV~vwEq63mNn+E6OLOisnf|GW!p|S??Ax_ufnR?sC}XulWPbmx697v5LKATJ)A|&X z{m;h_u^c!4f`diotSiaFtK=9~l?{b%~-%eWTp zAPH%xvi#>(U*+I{w63(cFF#5YnEGv_AU{Zdu1(pI>m6M?hEZKi;EsGN1*W-;PHWZ! z(BL5yU1@v#>X#(XB_gV`XXML*?T32-`B=J>3l(;Ru)fv9{TgE+YFZ%+ud1zg5}2&E zcVIw;w{Ra#qu{4-lB%S}`BY}@DRk_#EILa5D@FY(!}`BuWfSb?$#X2F3%+~tK`z%j zm4;s$Q(2lyu)Po@z2XSG!ovX z8RWY>k#Qxta~X@aH9~@lKj#=1&JCN%l)~BN(FR6RjZRFY*9?Kag8bYhNl2KczdNvd zn$fK9t)aiBps5ZEq5I(*14<0Y$N2tB=>eW&@{=w2Wbq}rH5s7_-i&VYKSvi49*Jjn z(4fF%w>_?1kFJr6`8h5fS;U8J_i7$MURAnbx71%36(cl=9-`KELI`ao(6-&NSbib?Yp7h;=$v${1_E>>6_Y`O90<i;wSdC|2G`k=#vHilSScVP-!#sO2_|uT4j`5nyJTrr#P;)Vc$4SwHG{LtgX- zJG@Zm9&g6ew)D;u4V()At-)awy`df@ie-e ze#S_+PWLj@rXND6`Di=OBC0Mnd@|uqKH8Aw?L5df2`B)>T^HdYd}Z=pG}nRGqU-HU z`_E};yJPcapGjQ%IVe%SD{FTm{d>92Ytd2o2+zlU;e#ukTgU$9UxOfK{6PF(M( zN~zH46e!H7XUHMv4o6MQNuzR(FLH6%y#Eo2gesiZlOz7JidmRhM@&`JI*NL~5hknq z&^GZUDr>>GaEwgm3|ypkM?`tZrKuQk7H^7hqVq)U=ZECtwy)%88Q)u&>iOvcHWIWd zPP0MgxtV1ygTY0oW$lakYQ>K@!n%z1)3J>4&HkIt1VlxKB5NT8(Ig6PdB$h(1o%^b z){Y@bv>2CAt(A$JC@I#u=IBk6F=88d4I|4N=XpiV9?_fQ7YVmSyDsoWJmKMrr^HvC z%OKJ_dtpf+{|uP3N+9NIx>3_cDxzh5>%bL_;11WSA;XNUv;)SV&qI~>3B{)=x;#co zuuACvO1d7wNQS!&F-u*}pi#t}RCZd+Z}j2)5xSn^r#Iw)#1*ymUXYwvus>KqVqf#e1cAM)9Z!QxZQGCtMKW+>Qt+|fJO6z4%wcU_XkdA!QrFF}} z@B7HV_`2a!;7WbbhJYfGgk{5%A({&yN`W z^Umc{Ma2E{BO`);Xo3jErJ;6d1nS{-4fn+(|Cq#jS#rr#@NxbylW<$Okl622^voz4F&uThr9Jf z=9epjKdw2_!kCiYoEK8^&q1ki0!c;nbQ6c{-^X(!n70>@ueq6BZ=CSkXdhmkkwK0p zvUb}NXJ_3WveY$stF#;MBxPWJ*6_wzmk=-=gGsInU_6sTeV*Ix?$Y_B1Mru)1E2fs zT1A~iH#;;)`p5*D`Sj+Z2C{iMILRpWJW$_y8%r+a_EpZR+nb%Tb$_Syz_jy;96n7x zXXv$0sKLjqm$@$7=e~u?^#u+srQRH(>0q;q4ye~%@*JF|iTKX!VjyipWFBB#tf2aD zlVXdHM1M1}e?TbyK)1((5^#t#ED1uYqKfJ`S!-^m+fYHsJpb{#!0P&}sm(4im@qWD zHk2FP`8>s8Po<%H?*)}>6J3g=7qw}I{wa%_tKQCSQ%a;|dqmaNILu`Fl8eB%tO{;RCKlU~ z^fF_%@TO;)u_|4RQEyGB>z=U}oEBVDVo5+nvKjLq=@)hquhvgsl*Bn(COgRzM_7iN z`<*a84Xf|~B`V{52Fs;Aln{9SN=Jy0+?5AUm)f29t5(vdP96_o&nbo57UEogiPp0=D{8(Wy)^ z!oK-pmQGZLXW+&hp!`y%r!b+#y%GJ#3ewQwXyB-}+tm@8kuT{38%liRc2-V^tB!zH z`<|7KnkWq|>6$Vn8X>l)S1vdtRB^!&}Jsis7=3_e0P<0c6A;e9$}<8 z?{fX&%sP9kk-~aDnTK(DFt^Q9pZ4`d$$QF@#z+Y+^dgNo3vq$xEBIVtaP1KfAH6Da zdKgCaCt7^Nf^EFZ?x3DpX1E!lKOJVWvq`KvT^O z|Fj$U7RBL}ZimJG=`rplOOh+0+WdYvckxEm1$y?%6u%BJ*Xh>vu0_jHJ-pc_ivgPC zP3pvVaSeyz+`j6@@*8{2H;tU1IC|kmqC#R}7=kS)vQ!(wHRhmdRshXe1_rZ8)aec} zD3<-pB2FGN5WKqG@{6WK91Z5{zis9?WJ&T6PC%WySQtsq)2wZG%zE5<3it?p@Xh{g zRu8_MWA6t4?Gua>z%_L0?lSSQ7%kCWl{?KndYg8g^B`)sRuf~J&jq)<1UukbFEq)p z3)`IAwKALS1F4+$r8*DH3k+|u$+1f!Sep&W(*sjS+K|1ghG^hPSeVvyh{jWBmabvb zPU5?>;tD+l(2N1AFC*B83JIo2wbT z=RzIj^oY2VUK;~@FU3XxZ}kfuC6zlv+{R6ysfm=ZY_*lD$qYa0^wBVlk{H1Pn*$j~ z9$oq?o5`1~3YOH7^N_6=ODXb=V3cYoO=JfB$#g43Mny%`dcDl>UOO+C^0YZ&d&~g6K+YC_QWjGXk`e_;9#HDO_z=+f?6YEva6`vxww?i z{+`_NaMb@xTq@JNJirCT@(N)-vS$Iw{;ypFe3Z&>MFim0MINHwK+W}E&)9R^l!~F& z=zaCc?%GzbJl!OB^`rd16>+RRxj}cx8~?IBwdH(bwwACHyNRW?X54S5d$M_}Y;$R) zOFupu%=8}C&8ABHPEL^Ye=YWZ?@d5LP2^Tn{eE*@0baCq>ov*67o}JrpgyZx&S^+$ z*ni5W8oF_LG%k?|C=ZJPMVq6^>8;cmU~5Dd`Wh3oU~9CiG^X95W~yybBXeZ$?u2Xp ztLuRVQR3CD^6gTmId8xnZ_V-tbTvV9nY4|#+%$tY@M7&1Qw>4`>L?M}dyf{@z3I*P z<~~iN$#`coZdschZuCYpAN3AQYA>G z4OIv!Oii&c8uDmIXfn%Fhud$nnKbiFu+L<{ zIyB@aoz+uSjF!cem-Ai_@SSc&rX#&5z0UDWH;Al8yB(bMQZe7CzR8s6fJz-mHQ!z{ zTttlf@m7oX{M%6ic0N{$~gL z?KhVJ{@JN9@OM}H=a;4`J;5$k9E7!J=Xpz~sFPQ*TD4Olw?P}Dhs|58MyE4g?4R|o zI?mslNI2K&UVkRmaH+fQ1>j+v4wg~V>UGu!Z0>Lhn5e*H|4)6Z*0XR1K~G~7<%`W zBCE0!HRZ@vONz{3z~LKQxC`CE$gcebIo=5(&O~L4C<;oQUEae~1isk-+G1%&R99O> zlg~auLSH18rJ*2$zcK30H5gCJiqdtw-2|K{soc8teT>6IYt#zjO61i1Vc+u+9}e&o zZolAjQTNVa0_s}W=m{C-l3P^OXI92;iFN;CBy$J79qT2exp=p|y;^PbGG=uDtIu*( zwux2MKmaJgW~u9y<0PIp@EL_41!5G<6%csL*>t(Hg0#SuT_xsts7 zqUuGur~eBq#Jj^%*Z@I6lGBVq#jvjGYF;OS%0hdnjX?G4IXN1h@6-Da&OWTYZJ78* zDV7a@xZCwUwGO-1S*&`S(=-`}^OFXziwM5wOnjl0Tp{#|N^vEaz5m3_J5y*+4yt+n zDet?+Rtf>uF1XPCObRNMG1@*8cLu~fHiHe=pgm5F3w6SIk6GIe6MRK)KBXuLnh_xmC$dCUMFe+ znFhg0x8Xj!Y{BgLmTtd@rK5q>{%vEtFCN|vEvI9Tre7in=+YZhzi`qDrF*z96 z=}=$H5uIBA<31#Q?jyNWSzKte*Ai1~C2Pv#j2_(<8l#2cokG<3x>vZvp*;X2>V&_z zuy&)+7D!~gS7wl(<^J9hfe!ps!poDmQBik`QELz~Q!w`zU8}FL5au0$huT9mtt|UyZR+M4 zynqnxn_7A?FXGFaxS;1i7+q3La1Ba?=xwa!hk#@DCs%j6+)k!nS;|z8pABq+@KJ-b zt&$qGwTfo#Kd<$>oIT~tnY}uxJ4v@=m_BY?a}nxF(a6yM7iRl;9@b@N9S=sp$Y|AP z`uM2D&AWAY#P>bs1IL>Qr!lFv`I}3z5p0HVAFaK3Ub`U!F*d3qkVFs8HWEFFa{Z03>?0Pw}lUgP@vbU@{)GSe?>pf=q|3sq`L@WgcV88sj_ zbP-+yQjK}I<}6kmNlu5WzGOd_;t#v$hHn;7EZXHxifdrS38lx}%a0q#6*;H95knqD z4N#f;)n0P(AN8>Ji=@#G<1KV@5nA%j9ecGc^;y{0kw$ZZ9}Z($`1U^2#WoI&^X%6& z{0hIU8YhN_s32#Fvq&Z0+L?W$8GEKBINegubJ8KZp;+0CC0`=<;?Zl|N^c~j z#cqqjp@zjg;wk*x$r4_=$8h!eGGVO?C%BLM{Kt{Y1hVXoA`0!!t@M*+R7G}iKCgx3`0SXB>}lZ5Hu{}9!#tPC!XJ}+$Jd`<_;J#5D8 z$1DKv4Wrc6n<}|>%^^ekl{Y4>u;$~zj;Q-{1d92W_*w{q?GFy^Q#$THnX_o!3gDZ1 ztT!(fAdtFnWwl}t(W_ru^J?@xSZK?vFRl{%%2W2Wx%`ypRo=a+ppWL@itmq2%)=uZ z|CIp)=tkTt+4<k<(x&h&{cUxM0 zZg?lZ$E($}$|3-u)|^n%`fe!+=#J>k_tnPBqm>_n@!Y%buaHFdH~QRQl~oJh7>sk^ zl)r>w003Gw)aMa3lgbNCJ9eAgR(d!+)P&E#vF! zz{e_YIS$?)ZK$+}#)Dobb>M%-pkdNsSRMvTF(rc*abK8Dj+2VrS0pbZcg}wUDV?!@ zvgm%It@$RG9$)vp{YXcmWVYk^clkgi@_3TIFMHhZrt;nh91SGeyr}-&HQkdJ4-^Y- z*@x^ehG^`gqpr{2Uxp8Q`~Xu9x)%buh9Z-p8FwT)e@u)~DaXQNJyj5*?pc3 z1;B}Z6qRX#-z~SyG(&# zRhWtMwb^V)yF#bobM#}zgIsj^G4<{(+@h@gD%poshDU84$csLwa!@{l*)jW>8Lx&p zJYDP)cf&7exL7t)PfVk7y(Br4HgM|{SgJ4cpAO}a#|d(g*P6}4+lu#JcQR8fa^ASD zN4A4inkriiOxXM;&f>d|tLJ@GyH%MOdb<~WhoUDB4lFIKCB8ra_Q%IsUbF0y!1ayW zPIBUwvQyvE*GcUXXYXxg&~~kexoa$9#0wpMUMknojQafQpgKwre)#&2iSJjrYwB_@Ln(lios;{+ z*w;%uTL71fNFB;S6}{>z=bRjA_gP6X>M04H#uCoU87t}H=Q>g9zndJ)yr8!Z+BQ7x1t@r_1giEzVGitox#UUcE8r)rz@_jDc zM3w|Mr)rr&%&m3J_}xj|Wx-3&uDy{pz8KExQoX`Andv6fJfNQS=8<4H5{V_vJKy3k z{_17zC3rr{GT53-!Jg}mWs+rpWxY-s?_5ra8opn&`juKeJ$GkZU^qA_Ufw*(+g=o=8_S!r(3DUr5IA%Td#VJZ zs-Asa2D)p0r5T$aF2y`mi?o|s(npVZ9RE0*SMipSM@Z`;QsH$q9N;vF!Z4NcX~&bJIL&SXi$Q-hzF?ulIj1hMt*xtL zBI#`|gWXi8qHCd3woUAK2nak}v|GH>1aIN-vK@dDbG#oTS;6^`;KLTP-FC69di(BG zfYcKr@ zK4%vbg$&MQ?L?E{yI~ZMK8|$T&hBcS_7}QiC^bT$uqF?9F*9i!IrKN%weA@=Yg#4% zrBesc-!yQ$TTLB&yYrzDd&~YW5ox@Vntr5cgbq)%jQ!?WcJ~ zA|6aBjT-K_IWG?;CMoQ&Bq^Np$jQWNw5tb08T$a3eQ7_KzNB9Sd8k}1Pu@Ds@c|Ya z2ZPmzMmqUelgCE68$3Y`6U;Drmq48u-B`XSWccl)Tr2!YIpT2zV9dOG``>R}c*U|H zDJ7V*8#!;7vvf#ewas0@nrp>@;DI077W18kNgC*M$tB@mmuVGfURQ~U-hA?$you!~YTb0f88g!-8au;~3kjGbUe!Xz zch@|cu9;&$2IBKgyvmuj=Ax>4n5JkkFX;@LGE=NhUx;o@9xpsCW-cPSWD%#wFGevf zn;XJk_JyyJ4oWjR5Y{1Pz8$$WI^#SSe>(;gWDb{>9pSPC7 z$;B`{qBwnjdo$IruqSe5JRQra|DNXbWy2o!9-1^IlYIx{f+~|j6egk1ym6hAucS5WtpmN?0ni|E{1#H< zA@LsTnP{rYs|>qrIcnjq&JA}|ik8v^0iZDS4@nJ_!VXq8mr5kN_C+oVdP!%2BBw6J z(H8aZlGuaagmoZ@8$|L#hnk#cqa5**9<5HfomddW^;-CMdFlOc?wnbN+M{}v=&Nj} zwhK0?xnOm`7%a}hFZ`b(+kcLzR+tcJBZzk;*f5Rmq))vOkm)Rh5aF(7mG<>F)vntr z``}G|Y`6{C(Omj6z7#b>gsixp^}0gbuj0=iVX!W6lPM^Usw5&@p&wgyc0%CVN7BC7 zkbwLs5s>~+=2Y9uUqi^+{W6W8S+J3dX4n~Syc0mcAb&%sc1P`N-T|piqc8F(uXqN+ zAmW7mouT2e8~*6E0v0_KHYCit<3CG;<99$nirV%Ic{Z*V10^Wjjt}4EiDP`qvr;FI z6jTb)0muLgE>R#$scSY}?^$9@7xVHQVLhQ7(W(}n- zcC@r?DdTU*6fwi|4<$GWhCL`qfwH^uUf91Y+s163_Kwu}Sb3KEIx|#d^ZBt#fji}a zf<pMl1hf3-3M4i?=RDY zBJMQ_yIWvt=X0we_n_gLPuLP@x98cyDWLc+-gmj5|ILHq{BNm$R{I%0l=4ZUX-~{hKE`@SI#^!d@@a%>PG&)R5sAA!Nw&&i(}OFugVeC_S4s2*qacO zOvApqb3F`CKyDJQBi2L0Hoa7v372*^{+N$z_6rUiLMxbtCko6ZT@!+LKN4Cu8B+0_x?6~ludM9PfQ=U zNBwLM>JjGBAzyPymm~XA5fc-X5JR`?pt|co;zDo0Jo)=>I!|f(U5Aw529U+e`xoWkxcag}Y8&C*eZ|pm`0XFN1*cRg z2Y-2TzQwQ5yX8orWw;@h1#`fnnEq5c_hMP~>3^j|7xtCajtU!q{p>%JaA7BGq;V;xCoC?8L;PKgZpjp6 z$1ndWe_4(xN|;uI2UW?RvC{<=z4WC9SLG*v%1F4n!$}_+-DBy~gnDLkbnv&L9W8%y zv1D1;J4x7uLuzcn<)pyDb+;ftl?)-j`WZah%r9*|6j6V~R+b%ksZnVN9$^=t#CTzM z*Ud%o^mp%?#CTqZy1`bS4f1~XA5K1xMtEoYxQknk&gwGpr>0(!s8r*XvbDc2{JRDn zkb=et`}2^mDDUm#w2rCo=10tB#h?pgyr&C)0FRi-A@2CMz2Il-9*tz|pC$e39aO0M zBe~bl{=k5dzZ?ClJ2Hy#&+ewt1W3|Wep*hJRCpR~^Tz}cL===I{lyppz9Yx~2CV;2bihW(!BK5KQwOy7swnY@F}%I!HvSaopk4lJ4?qiSo`T;Xr)Z_osHn_7 zLFEf%rWy34FEn(o{$qo4jK>z*d{VGa2v;f4Ni=!~BzNzI=G_%i!Waz9n6d z%ju@3-DCsv%Vpfpna1PfIfd9hAneAQ{VBj=MFdQk*8PlSD7u&NWN^nC$g_KNZigzZ>vZ<$Mtc19y|mgh%Vm^f1_V@zWfO zEUXrYqLfcHl~u?M*EU`pkdA-{(zvNB;?#NMQZpUQ3~pCPQ@I?^EBFHMM%?gqQ~lc+ zyB9va%K6#S2iD!oY@*$ua1}l|gr?3fxSn`;Ltf6UtJ?@-IB2H-rV0u)r@vUxEVCaP zHl$S_aKi2*INzxGGa}FHWB@{!1g-9cXF_JE4gCx+q84l8ZqBAvl`y%m*MVUl@5@L8w!9ckH75+bEl0|lW-DFsfxcn0 zH{NeRMXvvH1*YZ)6iU)inNq@$`eW9%W9rmpChMf;Lmao+kTV7oF1p4vv zT3Hg>#3ZDk2yQ%1YY^;Xs*14NmrE{{++-7EqG;XtRKe?qXwM7TvPcEZ)(k9~Do-A6 zGzR-U_i&i6TX1;gv^D|LZHDsQ$LxY71l*P)-K5svYQ!HO!zLXz7z;PXsEOmg^NmKA z;1IkEd4K&a$dGTA#rUddLlNOCKZ`1#RF}w%#enfuaR=|L2ET6C*BpW)5A81>00NS?2d;@;r6!yI@dzeHV;qGSQWymFGi_ zE4BBylo5jC2NRe~caa$VX#eb+OMm^<9%*l-w*%?aAi%hPvcVzZBxAk<)AZpXs|P~F zk|?0`h&!b(TFrz+i)Jb&8{*hSF4KYPVP^<(S{G(x4GobRFZRznc&9T-55D8hHM~C2 zJ2>dD0PDu9P;M=+d=aAGxB6Nax{}GQQ_LBop=A`G3F6Lw59t0aJ1-Mw4FP5>(OXRV z>*qrZn=maB;S58y4@M7i>rExH9 z%?9;SCV%<9Q`quFn;zmR19for}0%jx?q*Mq8#+s zPBuW3IYL-}3Gh?0u_5eToY$(3V_Rjj_?Hl=B9cIfuOe{4 z_UG)ykHI^4*oUlBEG+iGt@%5hv|>s;RFZEcMnf9s=MMmHIhJ_# z?dx&sV|D&7eFxR(;pkZ_H{v4N)DjPcbxw~U>BqHF;P>j_p^q*RwTZGWf^|X7_2v#U z`)Fi#7W_)j%4140Z|OCjNC+1E@Y?aCAgy9b5w|LfotJPTk+~ORtc!vFuepC|U^yB3 z&P2M6sakH#kXH5`mx|i~dGYg|^r2R??S;2PJ#WT|CJHWl?N2m0m)~a1UoOH=hVrsp zI?5do)w+APK+BKZKXpu1+Mw)b zh|9c;`S=a+5Y83bGuB5d^az{ORyN0zYss;V{*SmAon%s@!!ToI*?_2upFcNh=M^3JipdJ<{oF{USg+L5O3j$&>%D?CbHXXOH!b|)U?8eSK01&! z745LnU66K}iCSFF9KG}KvylN8k;Ht?rNY{jB4JcMmM+ikwtyPg0*aK;M)BvQAdOU6 zzgDf*qQ#C#Q6(a|ho??H$4_Oo_!@Wm8C@16(=kq5w5t_DGi)QRoxPqv;nC^i%nt2$ z)k?)eVrNWIu2}PT=4N8p$jr0URBf0cxh$17Tp#z`4Lnv6gk!!KesvqMyHZsu6IO`- zp&E#)_qAqub8TBo6ukKzbI@3Uon!i zJO7Ubw-CyDKB?P{ zF6A{k96nzV@HJv|d)_ZB;rcI1q9!mzFTv8p}ylFK(6?7 zpuK;7-FDc;(>Om4XuJt9Fd)iVpJH-4ZrF_G$~7c&v#y+##i#i@V}qw3LS7<2>FD@+U>d>o$2z1V8FIm&JAe`P@{^kwOPU zWL0UZzR2OSsd#FPXyWmP{*RP%=IpHpRqf^bz&Xh-V7dJ|l9G>5l{9%jF`(JS%161V za6+^!TxL8wR;(3*a`KMTZ{v*YNTrnwe3~S^rtonun@_a`qDcyz?gR$F3YCZ3h;zxN z7bp-vUs6Opz@#!wdX0{$t#;)UnY#m^82%pB*Acf8mNpCn|)Vb*-~$aynSl;i7Mr_zsOt(Rxofnbsu-FU37 zB6yeRlEA^8jsgzN(AsDu_&nmXxyYZGaOmJ4OjxF|hxQ@3dk8O#3{}c4dgVIMz z%r1bZdVx-SyJ2w?EZL_tXVV$dicl<{rJ(P#fXRWyP|^SiPWEx}zRt>enmpxefspz% zm5>j&<|gu*pCIP#`5e|)tCy#Yma*^gO^x1%Zsgua+v7VH*XmgDmZ`ZS>B=csQL9NNBypE+u1zeBHe&6e_*Mr z!ux`1`xl~b86~~^)}eRCnHqERO&)JE=Bx*BR8Ob;^9t&V*0qPWD>7o8FOkN4AD(pN z<(PSr7&!gQiJ}l=64$gzJP|yVpm`J%Tk%zF5s>eY3%1MHCmgTF~-qr8W#Rc8AO(Ibe- zC9!^nWSj860kHZ#4-hiRVz<=!7?bmq5?lTdwEmuwd~#Uqh!h<1{H^^2N;74CcrSLk zy=(AWM{4%2UMkZu#;}c2>e3?V7I^adIkwVFp603sEwZzZ5$N8h-$TFZj+|*KEQ5ho zqJ#i!HLX@HCkpD(+~w{*!2Ru}jN#dWws|Q;r0qAcLM^Tt0cQsshgh?54+2LzOJ`fm4Rn+d8+sH+2@c``` z^cH)hcF)-#jw?rnUI#r6CLk_J@yA2O;Aclu+Qa;MOKf^UHXl43k5wV_BtzN)-s&%j z%2&3axA@SW6C}9K2g(#kg#-%#pCG`$z8|l`Qj~PLCF9&Zvn!~(9~PY2(`Sg?5TFFU zv47;Gz$L~Qqu*%}y!o6d++^SCe7N!=f|)^j3llhSJ!4;N5&rHT5DGN zeX#~Juk@TV^-0;c^4aXG0g14d`UvtZc`^xeb3;Y-^Ho{s+ebbRA2z$aWT=MP4xuWN~T_SI*436VEV zpW5Ol4TdM8XY);Q4JK0n@nWZ98KuFtZM#ZbM6qTwgWfADH|9NBCxqep6GG8#d-1$A_YaXSXLIlt`5 ztzM`>9gLYUg&%%y#8n34ykP@D$*!!Y{qoU$1 z;`?TC#od;;PwMI(`q9jaSCzH4bD2#PnjWtbLmNb9T=@gTU#R;)m=>Iq`JNn9E&pa&3{TET75uK8 z;~j%D48qW6Fi)P3M){mOVM+Ew$~6^-%%V#;t-(3^`P&p;+@$^&&Wl-J9`)jQ1o@#o zI1HMNzaDy|nR1bJI70r)IR12&R)X`6&G#Tb!HaSP|j*1B|=O`{BS3cOIdr192rVjuma)ZtH z{5kgf8G0W-IdmxKvwilfo zd1pFZ%rw0areFlO!L)(RgP@ZkejHHD-n`ZP%6M{OpusbCo1OHCm^ z!~9P--+$G`4ojg1NpQyaQR}Fd`owF>63W$?NVUd@ zUQDPqZsmsF&((mJfX;gu?Ud%`sHQ;^%kfC{`;Hn1Es(h1GHGkdu-qVFFk(Xbk$zAi zDrWsFPoCUk-v3-@$2rJsFK2uW0JMwek7VJGsrd_FaH(pX)%{iQ+(IDlIRCyAknGGC$GNUkn$=2xD)dBSb_%(Dp$MRLT;WS#Cz z;t5b*o{hv3Y@%{;%FO-f;nWIGAF`tRWwX>0ARM_ndF3qxk>$ila0Ntdm0}(TGvOgH zZ>yE&95vopj&{$Qz)#?frk!26R@PMzEOBn&st!>#R&}tN!Z#~`1ooN)FUz472 z3Ff+wd6GjIS^CVB))viP4Cgc$0mOWRq?@?|Gw*^YE%&Sfc}8sv2>fNf+r-$NrJZw% z2t$P2vaDm}9K9$Po-(4zDurG~kYHS)4_H-n_>0wDklt$qU>^$?Rh=&#GGTk#kEH}h z*u;L=UPfH+Pgn3buB4?I`?TAJD|(Q>qdBkp zXpkbvKUFmWIg9fKV6J5K%_V2YVry4YqChw9u`Pz&bx){`_BrsS?VY>+uyGHlPwcc< z+|Lycc3cC(3we976DF}|aOq>K$8RiHGYhBhVL>?#-7 zUTEwtq`PFu^J_szjwTJCu6rC2V$#?HpV7U3nT)}Hc3RdHS47uyCd#6jZr#4P6l>A0 zx+;+gW!Ssd1jEtPhBGFCw#BUzU6Zd0tNDAO4LQkn9L2qSGN8&$m7Wycb)dUKAUzG@ zj>207!okTlf@79TRz8x4+wi%_@Vfw@naw;&azdtngNl6_40Mn#-sHD#& zq(PEi-&Fu4e@A|Wdg1TQ2aJObCb|yv2BF?S)_)Ep7|f;GFkkK=Sd*D4CJ;%5Y4|=V z4L|6Uf}RCW04UlujK?yt5@HO0zB-p4%*;q5~ zib0d-0L*Qb73GQ6Ox~dQfZNEt(9%9Cl@Y+!A9whb9QPbH={+;A4@DzA`=rW*?%p2i z@*wiD+G*)nO-81()v)O?BWTygW%V|SA=@3x=#rP1Cb2%C3Fiv9LWI&TNF2~$*rj`E?4Br8y)jw$ z7R`76d-bcg#1#pXExcKkr75)iv(@561J?D{Gd+jcpz!KZ;Oha}(L@h|e~L(^XK`op zn%wW4vRtcd-vKHAx@HXE;{DoEA4{hBe6&5)6oqTP-Ez?I0~Z2BBAr zVZ)52>iy29RE1*9LkySrfr6u-Xj`}^(gXZN%F z=fm9h+-J_5Idjf+t{K2yQdLXls<8$>v)tE*gk2K+`d@(7>n(ktxZ$aqBY<#)Q~AD} zMyHRd?oX!#12~|~tXQo=YPqc=Rco%FN@&z%0ry?VvJ#^VJ$86x9y=6slaYd6zP-3# zRLEu^=I3o{o23-0;Ov1nIVk6+3zd`B!!e+KW_9}ggO%^+J}(oHOdg6lG_oGNnAigu z41jQbz2k*)0Bjca(r=@84uyn54IgPMt3Y8Sv^MlwdXA8Y>%sWl7YfLlFtJIs^-=N+ z3qI<_`)|iGzw~?-lfZ^ZBqQ|`p({X49@Qs{)dw&JcIpG2YHmN`Gc8RZ&v;YK_&VoQ z)K09P_h1}(l*y_g6|5&NhjG&jjw~8AV%Ca;0n$K)AZMtXJqF@?K*Y*c+6Mv@>>&EJ zbR#BhrYh#Kx2!4DbOyx#MV6eQ?^%IX0)I%!`cUh$+7ksRyIM=E?cAz}j`KGb)MEf- z%nw&S5J+a)f5w({@?5YUPX$Z!p6+`Kc1=tIR_%xYgi5xE1bK_K41jVC;EhUqcx;iZ zfqYFtMR%p}uAOYugc_@EYYs5$=zpxm>(gE;8iQVO%pqz(5r`l@fw-UP<%-IugVL7h z6#D;(s1`qT3Ci@&ng?oX)s?+-g?Al+sIyOXTMw4{ zC`Fd;g%pv1T|$Mx`}RN5ye3^sOiWZL))U}NWWU%BJRCFa&djaG@@>+LQgu8p*6N7Z zUm0Mw+Il}+O!@8dY>^!Si^gg-yR`BRMWRfmtohQ<>FU#sifIq*k5&7F3!O1YI1F3; z=~AytQL)D|<#L~iM8D_R^uq$=al5O0?P_m}cdI(?x45hI#P4~9RgN%xNZx$DX_R;Q z9TlRIqwsFsk>u5txswU5R)G@=?M6M5*NUf(5A|Nt z1Q8Zo)s4B6NYr{5-ZKFh zoy@?OO5X`Km*P;~6>4`^U!1H;@PFg=KiOX$0>4}E9dRDHVjEQ&w*=tIQ54r9vF;&1 zI9pClsC=WBbyZcq0%m~8*`m^2)w6}Q_ny)=ea@Sw#@Ru6UEIPf2X3jt%#45e=Yzo|8QH6_$}uuoGk#%;y)i2oKoXlz%gG|+`iifK-dY}>8WYoryv!w7Lp~l zZjBOOcJogP@ZRAtxYct2K+Q7^bs@{<@KLnU?!d|IH+{5^1?!n{S(CvEPdmL@7EjRV zys-ck$x5t;BN2BX6((PN#so3aE7Mge>BlQUF;And`|0Gh?VL48MmRhYs z>$hRp)DLb8KwG0hb;?>_R%73)45Ug-r7R_~NSK*s^=#J1D?0X<3S$~UyGc6MvXe`3 z!y8QpvFrp$auon69w$Bwr&ma*paFs5tpu57H<3i%D_72edd~pL+rIaK0Fe7+lHJaa zW&fs3<3^l>>;^)!BE8Pf z_fxgb%$6H=pzF=@)g0!{%0k=2ji#n1J!`T0YO8IclCOrNC3?2j>rTtD zv(FP`BDMsk06qE2LtMui|7Lydqy}?Sz)_6b?RM_;;&ODyv)XP}9bU{+IR=Y%|2*BS z2`qc-0UsO*7Rp=M9H~^ql;`jAAo7n*Rc+U4qcX2v=fk&0m&e)0;IZH9;qzLsvMhE| z^hCrh-zZ;2-ag!%+>Bccv2~rtm00knpMa)_#N4_A!#_WLFlxl^cu`q>hQxkG@K8AF zb(8Xk&Ja**_#1Gm;t={G+h;HmT2Sm``>fc(7j*!HT`5FQ`!oSS ziSiD*^Q+nf27jUx!srdAL=l>Y!W2oR`jVd6VXF@Ibx&Voh2^G7(m(}}?nl&{j))fx zSqJN*nX)U|@zx8Qj+dv~vn}&a!@|SewMc@&x&lvpc6pzjat6%H_FzqdiWQ4iufI4T znpXb+452%QUJPqwycdm*=ED!nVh-StsPC=XM6|TDT;AW_UF?>7IK@U#e z=ffhmz2h=lNlMxUjmt_kF)FL#!OtFGhj~?O6<;m*&To#QG{utm{jsW0d}DV1js|a`)M!!r`pLv) z=YLe;@@&m*1)4wG&3^gZcX>Q4xIWl;vvRkq=3+k=7(2V=MyxQq-O;NEZU-45u1LTH z1YVQTX>7A!G;CjHnpzo!73S6iMZ{Ngk z3r`}CDrnp7f@C^xvh4eo%M#OWCRQuHZcPR(uZcFgC#4l zL!VH5b#fcG5@co=BUQ8AU=+~!$1^;lQLErveTx{+W^W@Q5o}A%s`mCBD%#~iX$|3K zhPUtI7a^U!1G%a-V3;Llv(1gR8)d1ci(PL$5qa$7O#REPwr5rbGO_7D`lZE1Mv|iK zyMb#?^$!(x>RA2v4ksNi6E5~hEirbZDTfy)((C?)FKwJ7z`<}=6@3`B!TPt9d-4LV za21JHlZeXpXcZ)ERk){Br4DXj!g0#p&0LC`JzLATsS!Mg&f9l@;w{6S4yiiYPrGAi zK79OADJ^BzQ5ZNmv78iq$Fpt0M7I}Q!uTv!pHfBRR{GOj?`_p+-*Q|ZufWh}-Tm=q z2aq7*ErmblZEqYgzpsTHUg>|p>Ob^fPpSrJShovP7?Ai^@=Z8a<-2i@mybXh7+b`; zQbc!wjHxsil!2~@2KutIkKe?DS_|610;^&78Q5o3@H-CCM*$Iv>bV2!!q~_u$nHoV zcS#qT1Q^-r3Q&c~f=R`wUXQXg@r}zoQ0i~kYqo+0<){6La8ValYU;TG zz^P4_`YRBz(l!GHR5_`B(!rt8L1$UgSGLRI_GCg}&@5zveh2RI!cctsz2HjLy`h>5 z8H97NO@8uUt>#q*KK{;Qoq>#SOWyRCs?}#;VPkjTFqyS7=i0@w*`Hvg=6Y3T>C{K4 z5WTzDA5v{$AFr^K8*kT7o6Fq(&$TL6{}EJ;euQ|9=?xZwDkJynkFSfBCu0v=Fhm0x zU8l!DK{uNk0Ti;cK}w7J*rMsObyBg$_wfAPr%#Nsd>0mxWq?H^%zftn6cggGxIP;t^$SACpvh;6*9U73x?maIA`S)Q(D#7(e zY$&Bt&ub4CaTUOy1(rWv5lSn3!CwA2!il;c$`|VUft_Ww%mz*2{Y>G*iZZIQS~ylY zq&@VCg(D9zRl%?iD@zL4+vVP3PUEtV(V#}*vDpDW8VKHb&3lZcqudZQ8e%O)b&Ns& zCD!H0*i9?h2X#q5;xV{R&7=(;J%;X|<8{L%J^Lr(EC|m_rsO|8Sa&?%A7ZsVAW*xU z#lK`QUGV#HwZ{St*8sO$%*3@@X}v2N%Roq#e`Z)Zp29oA2K ze2o?aY&sB=UP_?ls~z1R`riB|U8z38VbKJQWa3`Tx=QkgHo%Iw!WW&HWA74PyTzw_ zZuYa#GgecB@^@s!UawH-cp55Q?*ITqI11!C?8?ovq~4zeC+^5YRv>!L=%l}Va67pB zCUEY(P3lvlM#1b}V~In>Laa-D<3LaogBq8R$=XCE10GyZxNY_mlB!a6=#hIEq+(`% zp^tg2DUm}sOz_M1lUA`7t%##xrV(L$$)1Rfkscfl@gvtB%8u|8rxUGEN!cQ`*82cgfr zKhHUch1**$Mkg;kEbTx*z+}0hRAFH#Ty{8NJMe|*kq9)N!p)4@aV0&KMY&r)NR#U7 z%GZDM>44|x*&6ZIsDn)ZNqP* zYjN6^-x2{8sNr)p;3mCGm#KvxYS03-CxEu*fjI~a8$1b6^{E2Phv#(KmouOHDskz5 zj3D(>xGx?g!rYf?0|hsafj6_;<-)qS7*!F=f>1!NY!W&=3ORLP7$Z2P=619uPnbV2 z?zG#87Z&+vYta&gR#>c`Tp7Ar{oPH!0xq8_kTq#N4+tjaBjT{dQ()8i`K9J;o=VTH z8u%B>HK+vd@prJTqPS~!MkhahNlqU8!`{Eo^AtqQQ4Gd2vw`dZFOK^XF8ULBdRAWM=BkWW*;+n#oU}hwdsO*QmNQyzHcE-SqOSGae030);nS26Png3N*cUSA zQ3}Q%3dgfN33uDUvkN`Bq8(nL!#CUw3>`T8qH0j5JJ)NOYx0yU$A=`ba}+I&u?g!k zAM%Jfo47{bCHC&$)^=gyvCp@^{ekeXHDtq0+0LOxOgAJCrCu|4<$eCU#Dk-Lz7o|0 z72PWr9U^EhZj1%C1bN=99VOkMM@biSfNpTuctw&EVn!q+YJdW+s@$Cr{+PqFJ3?1H zM#mz?35K{ln(#bTJ)QI_tD{ zB#8Q?{cmB3;6`M!L3O3GO;B_QwtB5KQb*cB0- zuc(Dvb7ztHnrh1gI>(dijU5;@>t#n8~jJu!sO#0fm(?((kxWPHDGSPZT*yNS5-g7fqR2XcDhm>2vi zJiWhJ2+K-=EkBDkg%c2S#(f#ZybOMR{79;ad-&+U&625H=xtK7PgD*wE?@=TM$z6) zU)Os=vI96_nvEVuA;LpA0hv^*Qb%9bWV71mS(PbSCg zGjk?{NYr#G)Yxmj^LBR1#}Ps-|s3746{XOahS6ORYT;$EI}wgwV6i2OqI;@o^=`WA4w1wp>-v!vMZU zJ^!SIKYmnmpBc9*IUHArTcM7XZsKG-2k`IaF0rEX4Ey)g!70LdaEdUn(X<@H`)Lx& zZDYO!9xno2hOvjZwDU7gu!B%Z+EnWP>iloPa14((+0Vro zcWpe5K9R0nT}`^tyVrA~`QYq%$DR@^vaA8qK3tUITeOno!lX6Rf1y(RVvz-nSby+m znh(-U#7XSVctAJOBoM9EKth|Wm&)<gK z70=CM|5G7w^rOvmJ$wR=x&pbUg}I1kXr>=wtc=dRMq6m#>O_!s;nbjKvL8>&Mb;$C z*1g6z#Pjxvtt6J+EdCH=v&Y2tkivhpU8O<5-rz6Q8^m^avzxTToo{jW-Vr*;>wnJO z3kQgn21ynn6YqB5|DjpVX{3if5kQ(FePUB1qryUX@vhuTJ)Jcqi8><378HoXgRjz9 zWB#YSN!ec7xHEMb)X$3pLicJzSfO7yD163nv27H=?d{jJ&2P`7Mzj=kUJ+lUEhPS= zatn_3Rl+Vd)QTOe9V7_2A23on)9%?+WcxQxo0}{FeePC_WNXXwHtgnqt?u-*+*{ zj?M|)5&|sf#85Y1QreF{*d$-@BX$-2Gz|xbEs_8%CIQa081`cbaE$p!Ms+1q$F6Ug zPkpK_lsIwD#h!|3lJlb)IUo2I5guz4iy`Jkq>CZeX#T3qUxkdsFbEGyciWo**}3Ik zY6fhhEQ29m1wOGLM3FE}`MKn>wdK14qU{b~1iztsODEuE!oS3tckhNlACfm59sXbn z`CBxdaaeEzf&eKp6{K3;uE`{ytCZ7ZswtE|_yLAk-mPA)s3J=e+Iyw+du3k<_67b; zZ6o3#REu)s!EQl&fY(bTRIJ1z!6|6yR1NQqzU5TlHh@D^+&WDU^0x2wAzlG2$#f0n z00KqQuhol_aHoP!zi+mlrn)|;8)C;YBqlvdCm;lR3Klu;r4!g+r-fKUGR+Y&otPckicu8%O0pe=7~_VLX9`LY`N z@8iCr&&loKZLPm~dHp$4<@EuBO5Modx?9*@C*=v(4`Kf_iZ|%LPF8-F@I@4SHV7*K zzSr~ptUvRgt5Au*NWZb@ZT@+E@)Qok+*QN6+t}>SIXD-g=H?Hr*C)C}?%v`(@iaS1 z$)ukr3Ay3E-@Q60H;)nse|_3EU7n!46#~6AyZ_HIW-dwGf@ul1rQoZhMzpt=ZI!=T z*+1p@eGz|J8gfB$h1Z+`;xyltgke1U@aa00)u;r&FZq`iE%e0@(4bqYE?{T&zdit! z%)!9P9h~LzQynKp?9XXfX@!ku1Kn8*65SZSfGr2y>%Z>a3a&?T4sF+zsj#bh3j|R~RwT;NyJOj1zu4gVtkJE)# z-J3p3TItterE|xFt~;VWI~SMvq|c(yLK4WM@lo%iBD-L6Cz0uJ`^9l{W_jO_Ah7X1 zDQy&#CbER(qjDkW4VXZg7H@xYkDD_i5uGR6fU0!;GVXkXm7cz^qI+$_pyIevHDRdA zB7vNldADl(u#4pl@e9O0!cmnx?R$?7YbGv{++$;im<);UyAa%W(ZrK{_<#|n4MrZ8 zkMe00VK^CPiFxp2SnwZiAdsKL|A#-(AC~g;l-!1JSOf{8!;hBp``&>-eD2wOxdk!t z-g^V0Rs8*&PnYK=u$fIJ#>&q0S5vRe_oq+3-m}K@ycM8 z=mjDKLiYG$#X8*yM`ljWAo%I@oZz29p+9_O0O5EU{I`&pbP$3_{-+QAkr5sn5+sy_ zdhd^jw>%Iatxx|JOk7fu3tuNZJbVO;(2Nvsm;D0UggNQq@h-nEc z2r;~0QlonykXrP=K3|_yULMU>Ehs3+?K_dz^9Brs_3vgkgbJYVL>?n3O_)L+5@<&F ztJ#Ie6s@Jby`49sYUB(Z-xz3*@Z8P$WH#2t+F<}#$ufbQ1NB87+>tBX) z0@1wlmsM>|Ul3vy5#NG1rSr4v{V^@{t&aw`uD(*>|5rcgUWM3C z%usGYG_n8cf(-f4vatmVLO{>YZuqB%=UxFsOGLLJt+)O*9)d1QZVVI%2jbt0#jBv; zb#*E+knFE+!GdtY`{yhF^0R=m^&;8m{2_XGfvxhJ3Q3$rn z)ckY|7lxs?Pcw@oL;gtc`JN)}K3s8jdYe|9~fMg2q<_{JKbn=8KuB+IeJ*;J8gJDbt4K8{4VDX z<_=l``t)%60rsCh5tcl^l+$}vfBE&u1H!N0^jl@zv{Utc1f=*02*R zxsehEZY|g4ciS3@Z9y_Ao8WJEp@v*-a{uI}ZJKR(j!t1bJ@DpDUS#IHeOjMu#YrlG zZX4t<1gfzWZcgFH|+C}%K--chnr+z1s^FISV2U~{q}Ok0~cu4l!*?FQ@Qdq_g zLRxma;=0-}B){1o4S#K*+z-7L%te!Uj;_s8L+JOI2%8%5pa#(RNA^YQiXG^?@guVA zaKEN8qCO-vjsA~f+NNI;#Xn`;=ZWG)eL~l?&5NeE+=kMveZqOY)esEdw?r2(Gc>p{ z;E)D-W13Z*77~wjyHj;6^b&jI?R67Bqd8@3u_Y+hVLkR5vBQk=K0O}kcgGxk8{5+~ zQ-k?b=G2mHBRlGu$?SN6qehtMM(T|r8W)W<>y3Lu?g3do2>a`a+i^zc6Ks2@0n|0( zHV&g~Suum9eWx2$0dO_>kqK(}$8Sy?Gmj0vV`7cbcQ9th+4oR5%xTWc);ZRnW+M#T`z|xUI zh=$e(c{c0j*mCr^45R;!|Fd@SO;M~+cxctp;}GQ#T^q)hqur*WJ&N|R6jiPdo65UR z`XBgguvlJ+MXg(|hHGJ)2iQ*UPh?!Q7d3RJL=D;Y^oFLpr%>? zL6Ix2VlPS`?J7@+28;Rgn<$x5SK^14bQwb zrc`JOzgl3qf8O)BDKETdqD+c4TjT69nA)>inNYzF`yySqcn8m6Mg9as3xN5)C|oU* zNO+$!{mLu|5E3y^U0qHWe~#H<<1MBhe4QNHyN%>P8;XB4$jaLXb093)Y4Jjro_zsh z-^iwHB34Mu0u)BYj$csbUdX%KT<~D!yLQr^OR^PspxZ8u1`{3dPh-5_Xt1nCj&Z$g z!fKx2OqzCQBT9Rn&PKlXmC)g_$?{4ImCx3Y{8$e3##4~gm0sM7u?8ce81HYP%@k@Z zd?Be1J@6?cs#aF7pQ>|jXDGG}&@UWS*?!ydS#?|LbPIo!(gEGVF~d<=%kn5`A)0rw z-&d)R%G+Pu!NXL&FmDy@yr|kduvh4m-9d((KT6!X$HrIi>mgUDTR>Jo|9V5U61$K-SfpQRO*n@@}%u%A)Vf0X-YH325YjA!+Q zp8dq(<;LXM=G2gHBSXFl5n9YZ%vK*^|5El!YSdQ^b*a&yQ2yeE5~Q&7l#(95FF$@( zOXQ;Dce{9tSmRu#L>kVghJFDPzm-)P^H|_BQ)kqN%pZ0pLSQ}G>ElnI&$NF*!%n2K z30`=_+F?{mUQtELJF{LwRqlnYYHS6I{RERFOUaX%LORU>FI@vlP&1XD&z8b%-uG6XgD1N>w{9<@MbyQF+eVCep zX>rL{R5#n2$HuiE56a3J6*O6pu_Y?nI+whz&CGaI|lprxnF>>}*NU;X!D=09o9+Fq!^8Bob^|nIM}SH*!eC!!euIXW>(~bXB9D?LjNH-p&_y_tfgAH&OUq} zW*sZn@3P0VT{s{MF(J*8&vHJ*k=U_0aQvZ>SHN*4Ch!PyWks!7qGv^E2r-Ae-&0gG z@ZH`Kauw{m2EjMrntW2auWRMKj`T9)90X}Wh`lrXE@hD`zM3DP&f`WwWh0)KP{o1n ziUsGik@z`GTf7AP^v2yNQ$b3t5djRViLy{P#WF5_Gd8tT-g%VIh;}K zw{-aGh8r`sTiz!`Mhwp!`kckgE)J8M&5(-Sm6UY%pqiayj4Nbqn6`paq+YHt1+J+r zQge}HwFhysokifgJw8ivwUw8NHcszGZ)iAUWc*CDvHaF*IqrIXDb6^qijrFK>-<1u zNxx})1%?2%k4IA+#_Mx$k!PoK1Bo&|Vbbx#>Kcj~L;xll+Pp zLd4B33BBai(z!5DNF+xk?s^!i8#I;JB5_J0;HOdNN9vZ`X@DUoei1+d$n@BlbVR7k z9eB$5A`b>h)!os@1$u!EyaqOK8-BEaynGyq|DawuU(M}9GG?xp2SGZF2Lcw`?`CH; zVxk#MzH8>G7e#5{6~Hy^H+eS!V|{lyp>x+<_=vdGkj#VHipCppxYA6YOp)9*ZAO*WK! zYW2z4zESK&h3Pnbo?-~G8%)$#t+dJ1&+X%tv7VRGt(`n7FfN`h2?Hk+TC*8tIXch zoZ#@DQ686LSL=Ct)Ts_)uvn?lCD^ENu)nN5(jHK>)kO8;5i9@;?;-w*Q7)}r4wg_?>u(_*6x%m)A4Lq1b;HGiupW7#iF}v1cZwq zps*A*TvcR#wKmcI)?Lk0%wX$zMJL{!azZcB&@Rs30CEjKA9v>MwR-2%6fuo^uv!M1{UU&06m#i~u;EVsljN^J74>cCi&Uf3!*N?&GzSeT_6D{cZID3?+R;^hESWSE z9~_06g1wl$FwWMzN|$vMgrWRwI@wIkXZ2M}%|A;i8^y6gRdW6gNXeFwbA=Z6oxC&$ zSkCZkGR|DHtU1`zyh4^*ES4(vQM&2Qk3P7wtFTa>=!}*|kaxT(&UQd0ktk4D9R6<< zHmR~{cy98g8s_9;{s+8E_2$f;cFr`t;`GVrNGVt$jS zj}ur+=`Tw@%$RK*&0=QeZ#BfL-tip|IXEktuYqpOfHYW5YN-Y)Yhwa-F$ZK+dim^K z1Cj&s)VCB8#woWRUqvppLm673$0>pOP_@N{+KE!)>bS%v#^M+DM(&3A+@2Jwndq0E zm#D;dkzSGr6oa6ieE{16l)jiY;awK7iJaG&j>%)(aOqjf@dsRj|q^Q@H z`tnw#sGl)Xzwom;%rk0W50_K5Mq=N8e*Td4>G5M?w}2XamRI!Z?-{LAt1+=8m>@sU zk-~H{PliNRPo!h%6n5KIOO^zZGZ-D`Fn5K35mu^OCrjwDWr}xCT3MOcGl%dN=;0Y7 zjMgk`olIje6+}CXFp`!gC3tMf5KFDmjMzhypdYB%>j=6)S)z|I zUE!%-Azi|V!z3O48lD83#n6uORX%R1Mp53$yK{-}S$K|#6Pbb$QI`M)+3ac#Z4wSu zwBw7amsRDzvZ#Ji39q;G=qFWrxko7HHmR4LB%}&pJuQx6^bySCx0m`pcDoPcUIhLVcu-^&TauDfq5kwgO$0(aCp@Y$SU_vTy5gSn1}f zr$6z3TAEhLcb2uwR?y2CLEB|+3yGy48gdtAukfjo>V;#V;N>FOJHr`jH6U&z5I>Vv za%??RDy$exnErSoN+hw6dN9qNLba1uA==K9)FCh#x3`XIVV{^lcQgQRqpOe{O=S=O zOkMrl>uB)O6{{_c$ow>qYDi4~Ri(q;IzLuhhIJ#YL9bx4!-9$-zC(#^kAHP>AUDgX zKtXt?f(^8}K42cs=XT09+~x)~Ji8;qwUt0`eEe08hBf z!j&bpKlkt(RZmml3}Uq~>2S$A7!i=cR~4Bk=^PE8F_b9qUdBd5J*M4i3gh>H{a(qe zD?E?fH>1dJu`Cs=(nWn?M{O8#N9pV4AY}*L3eJ4Jh|GDou4lpvwiNiBWMRZlPX%RH zS+Mi#@;EcDwxWEOzDGp8Q&%g<2wELCF>uJ7S}bJ)-?zwSU&E_PmS@YwZD%f`mlO5K zo*JrmSOd__tT64wKwfP|>OeWwY@<%si@^AYc5L$Nxdu=WSoTQnE+fw;XP)SOSo#d@ zr<)^*rZYI@kiWP)#1!V84IA#HYa+_9)bsxsl0~J6a9NZVw9?oo?YC1kL>#&htpZ;&A&4jvl-iXHwyj8 zlewtVX7(noH`kz8v6E~pWc>gwR#lG1l<(T|*4sS2Gq!6>7RM$($U52tcA|;zz{=aF z-wz0S1=xjrXkpKOsTG@s)>z58KM=2X6;hC7E(AB`2^{UsX1?`E6+XR~Tm^QLWP{V< z)?Csnb+o9o^Oo`?4e4D|!2QmpFZeGKLzln%_ssHWr&nZ&hrOZqu%j~5G*2o4q=NEv zzW1|pRb40d$NN1OevryHJ0uP{F2f^zBClvLw|yKdMP(vVLqf+jb4Teq-R`p(LXvz& zQxfr-u{NCS%B8!}7s()57`bwW0=w=F)MG9|6#3ujY$VsrJ&LHRWz@L3z3~vn+A(B}RUNslYFTJFQYU zRfy(CP#liQIlEiM|K_Xq@p`+^S`=pbDXoK16-uI!Pb6o;yB5@efs>^}569gaS5+$1 z*~W87+Z;p=S{*#($40{#7&-^R=!{~9GLV*w(jr+*t?>A zXZCSXzw9Lod%S64#hd1HmwrAYD2mo`ck-stzKoz9NSB^}GVGRq4Wp&@c#_Bh%RL|) z4maC}KeDe>$o@HV{Wz0AFuP3vo+oY$muEz#r;XfU^4GDYNf!R7-AesNaRr0wK|(5a z_=+zfGQ0dAp!?nLiLqrbP*O@Hx=eiWM7L$wzX~f(=FtxI=4eBENSKO3&YWa`wX=s0 z1Y^@Z)L*Bi;b}!*RpGL%6nawUq((Z_301E=uCPJ-@X3kN;52B`_W}}m(%3f#`yK47 zThBMM_qt!QkIUK!b_PiFFk&kUMcFZs{I?!YADYxvvXn19ziK!B?Vx9I>)y`FenUJ| zxQt_BXF_G(0?BgNGr>@5-Qi&cyMNpZn&Rxz40iifvfM|kk3y=Itx7|+m$Ub>7eMso zklvr0h@T{?1VUAZ23eZ|`=dY*CoZ{}MuQmwEY^^8l8wRz_#aD?xCsc)hSeS$Y7nPVzhfmX*K zf-^;=nktu7brjN59MQ~3GK9%iKGhNQn?Jy-J~rW21*usYkp5B zUmlpN7y#IgVklGa(`%{5($HqJ?ka^*74{9VC(zi#_SIrY#fB>GJRJ7Cs0Fy{zI#4V z&v_@!3)SYB>WS9+<8~dvkeAvZcs*I?4BcwGpivvo*PVO`LO;nI3|Wf|6V4og&hXXf z;S0sX@_56E@I$gb-o>@pBgJCTdq^+8%ukO`FF)P2;ot@_89C-bVvIqt@;#zG=Dij*cdzs*!}nFK%`~_MMkmwGc(|zot0<9RyD=+x+>`WZ={k~S}X+y7jFE) zA_)YJPaR9rrMpirAc{p)ck#F2aOIP!4dyA#i5&$C_;azDCwO2~9iqPzSOy?4hu%?Y zPu^}hk8VB$Xi$wRTbWCI^Fr(qB+?{($ER?~KGUXIW2H_vAJWn%o@dX$R@d( zR2;5cUX!$@MzpEv_-^^ozIG655`n2%S5Q56yow3SFr8$w`%3NrFKTU9&Aza z$g%D9xlA_Sws70(An}O!tRG+Z&~0Aqhg-YAVg%Leb+t^%d{u!lSLd^3vy){{dufL( zY(C=354LO_8=a&{ZSFORBPPmq)y2`MnkW z7ukiYNMM?)%`0jiZM^#-ygoTyYES~Ln>ZG_=f#u;iza2Z;7p*Kw6ucbfgs8IJ1zNt z^J0MUVowSZuJShk4Zb=e`)}j|>1%Ha(puaVfcQchri7$)@~xzBPZX>RKt3`}0haaT z3Z3#W90qt0%4I~IH2hV;$n^^Q`M04ssVUN(O{bL^-Ba(Z&Fd7o?gZtekt~M_<*$D7 z?s8-V@vqZA_4}FiWe7pty*7`tfe)zS>U~fJ)9GVwj!V4~=Zw**Po?t#Ja#_U_e?bu z`;XwsYh2`&0i`E9e*R+b3?~LkpKC&{*CRy#>>^IUHg7T=n+}5SW|3y8-{3i~3`kImZ?UCyicZP*h8D#%KHM01#Wfnqr5M^6eVFZI=r0mm zg+43Fc9%MYs0vE?8~=ke@N)ClbC=#qQuA~Wf7Bym;7bGb_or#~s&?l&JdMRr{LdX; zefVc8X&FX3ojZ&0-ct@A`XUtaB3f&FIyWlqX=VE$gQPlcoUP~0rtn9*;{W+Ns1}B( zxdnWAi`WaPA3`FRtO@|i$GQq2UKxN|=NixWu@4J&oN17{-^th#(ZK3`AgnX+_CQ$V zyKQ=0`&1J|qR$p~y?dPRk{4~2^++e2n11s;`Jw!e^1w%2^9sCt4+!{ClA+_#4Vygw zwJ82mn(9fl4WztM_H=os-;T$1X(auYha(Z3NdPOq!z`qX{tf zH*B()mAbA`2Zi1KgK1t@oOPkoiV`8aI#>Pv;;Mv9MGp_ix*f9pDmqhRRx2Yi242OV zzxXk~Gy%SfWpSnm6S@wN{LHIJ{uk+l|F#VF0M5rZ9@o|Lw+KKxu0T6eikG(sl$<(+{${O(pMKUp z-lLmB9_B5KW?)F5YeM%^&!B0YWIp$sVgL*%5nMT;o4P#rG;?z3sE&Cm+n=|&$LwL~ zCHSs8utI|}540k#dENTWOTugcMZyhAm(|~OeLobNxVgK0&Nuy&6Ov%t7F?@?Jm-Fu zs2Lg`!cEUf+^rW+h3LwCuLQp@`TU9!V4v&1YTkHbH@zJEbQT|e5fP?PK4!1sD_y$C z>Z9J!V)$&ld^HQTXC;TPCj?#Lwr%Q;p;lsfFGJ6G584f%i|^S{sYs&Dqxf~;Dx{eK z>UG1fFhSlxPMk0j$swvT;8(htrB>vPs22y?`UWvv?D*tAKVDClFnszwcUo6F*KCEg z{Kp^TBukP1jO6pZ^b*HW62Ltf2)~}D`ym}cG8G>zI=Qp%-3<3LVmjR@-6MK9diMu$ z?aF(k*VBEs?N!#}IZX5rgaSlCn<`G!8=?O%Noar_^M6cW`+xKPryfU*XGe16bIPhQ+yBF|Iwt$j+jIGf@ZYiWT?>&1iU zOvYUen(K~rYfG}}LcvBy+h>At*DJR6qoyBYBD(_?~$M4E@x#CgxR)8)>IB|pM z#uk%2$pn{ZubO<&dz49Xy8pa{H+Ig{=KScY$*JBmu&rf1=*4dq4Ih>lz8(0`Pej0H z1f#*@_B1%})VUc7|D6fvxU@dht2PdlV4Z)cG3g;`53Yyd6AajTjpMdAyd1s^5~h&b zS1*VseL{NIiO793^^z`@@B21cu1Jt%5a8j+ozG?Ny8t$Wqx>m;yw+Yt{tZcsAFjFy z-s1fo$j(FkBESI4iBcxN+MnK5dlK+BJUq8FU1A?ETvGvTtG$}sbM9Ag6aLu~{(nRe z1iTF;E=gkU|SYX)co!x$X4Wz;Rm9K65c z!_2_|SV$h~UK>Zrx+HE_A`+~Wg>GbcIIX04Q^bwgFf6%U5iRvf>5TCoLB7=0X1q8F z{pkwt=&YYZMs&k$GPaVvevVGB^x$~~MjQtOA~5BuTwnO+_~%p(lq>c|=vVw>tK7FC zPRm{n5mB1~w;}If%~M~W?C~oW3@HE6=BxWmSDaF(rFXi5k%ADX0fPJm)n(m;Tj?8U zlsDhxd=CZO#0DFGFu6C|q-!4Df)Jo^6-cdwX0z!~{ptEsIva!|;tLf!Fwa`Er9yY40BaBp;6`BX zSM}Y@`LBMS_8vU|{)2c6=_u^Pr#8=-FsVrmxznEe7GQxc=?7ZAPc_5DfI#5uBOiZN z-5I$BHzlOLJMaBZFX0NrV;^B4@98S0jeWMKoP^QwuhTCi8{cEI}LBa2GmVJ zUl|a9$Q;HQsP~6^&DJNlE6yV_#-koYFIaf&*5JpUS89crPMhK&;1%72IKfYM;}9=0 z*XKFo=o(|*1+3dw7*|ZoIn?T=Z^4upK+Hc4DAx^`)hE@#IsbeK1Ena6f|9Z$F)S=> z>~XTjTh9p%0d$+8y#r&*q}!gGw}RC$f~WEg4}urBs<+C+5;ia`tMw0P>x}62Ncm&L zYmW!<*N87jx0;C7zxZv-)M(HgHTz3}btf{~iVxdD zut?`}%+GzawH&qT=tx+$a4n~2t3@DKEHD-7yOCZeoJ7BzIM6x9Ym#8S{11h5_RvcD zNQR$3zaGAO!n0N;my4{|pFk!fJY3^`+cIlUvHjEs3#*Haq1zvaBh%Daa*(CyWRP zanNtLiCi^X-V@It2-vR5`NtJ+&k0bX9DJN-09*S?s^@b7mEpt00@TGC@51l&#GP;z z>z;pcdU$uXMK3h8HSduNTZ`Kyg=+Gpy$CZId6zEkqy4dq5(|!l`>^W`j>1$}XBMFr z{IwdC?5+r-Yzs=A;zU~m%rvcx$=Mc8ek_$DkOvSpF-2Bhrta{>9p%f3k9%cG`y zesRP1T()=pLqdP#eZwWxNwaN>8@VOW=4)dkyO)fyUw!yJrD~y&ynn;`kWeLKYG{>~ z*ec>uq{kgerKd=!tz+~q!6qh4i^BKNn_%y1NLWQLtY&*tIOR0RE7@a)FiDs<=i2k| zrK$zx_GIW`s+Bh49YgJZl!3aDMQ4>L*$0LxV?W036h>W>*27q%k_8l&j*a}l3$3Nb z#k2kzE%eT>AG*(iW+6}VG|r6H8XT5lCx&aR83GCJGa}hc+SVT*Bp*nyFB-QdHEcAj z7~fXSmY!%mdbYw_PrE4AkH@eTZ=_MSqaurtM$b^PdzzHDiaoFU6Z|*1DE;O~G)~QO zT_14{h#g$jr<0AOqB0M^TaFiYwHbyXZb-dM9Kf`|MSTl**jup%E^RPzz=c*c(ymh2 zHVwyH;>!j?IO^258u-44S_R#|J~?o;hTUN{t5Rx4ZV^yFA3hQZv4r2?4GH;?CqPe1 zm-;oSKC$z#XI0`WOB-z}Y*eYnTJADAoFxfP;@7=e#%$tab+h=+<4OU-r9BECa2c~- zg3UykYLrDA?m;knVR#&`M>{KPnA{kRbR|&~O?Qw=KlO%POVZ;#4r)L*3c>g73ESxJ z@l8z2bwoX2*6A8!B(W)(5r_^Ta;_2zm#?zdHLyEWn|E&)JFLFEv=ZQ7x^p_=->-T! zXB1jFyxn}S{fn_b3njj^=tD~8a9+eUMVZrr{4(pIbKgl{k!7FV49o`{gn%QHLihsF zwH6%HR`VIT#Q`I$Oc*57fXIk6xE9qd2cH*sqzl!aJ*cfK78Gh4uFIwO3TEA8OS(VP zY|;~wcFHX*dH8ZZ)d1Qqw;zkAHVKzZj!d#Q~+2;q8brU8!=A9o{W2QTs;qn7D z7V+BicYI26j#G8E9be1J3l*PcEB959N#R9qazc6B5f0PnZTSV!tdf z+C7lOAtpy&PNKqgdp-C|f+E-CdZMK!{&@#KS?voJdQkoRzHmlPd^A5P>>b6QU zRpT68B4MyiGp#`jRMk^|Og%PlvJ2w@dYr(_%;=X&`%PQ1H0(RDgJf@TOyhNoJJ&u) zov5@pARW=mK@;r@G+b$wx!u(IV&#zh{H1(38xZl7`1}RgLr;2f#+A=m6-Lj(78hbo zP^waFXdn;p|5?urc?5}Xj7Pj)t{r)YPzjff3nvZct>$>1Tfio}?&rN}`~0mUdG(BH zWOhcnHGcup6B1^-+H_6 z>%OjYo$)%abI$FsL+Y>#Kyt}&$lyf79u!>hLtq4wti*V%8VeM~w2{0|Lrqk@YF);% zCdhJF?fbL11B|UE%>!j+9#LHXXXEiavmtRDxuyubsb}?x%0)&fTr;YYznh%ecFESO|V zOyA$wM}M^UDD-r2cNv)@Ils45>ZqUVOh~6veA7V65;unRh7d?&U#0%JX}RDeg(-wUHe|7q%~cV(#k& zawc{}0HOt_sOqhWTw9K&O|~L&%2CUp8#PX09jAFUn2bo3sR6nPj})>iH-lCw9F(*) zt&H#bQLb1-lnuVjFgw;5evj^l9YhXd2UNVDTl2a=2!PX2!@ylsSD5ccDP zjp_dEq4vwP_*b|APpU^I06>-`+I}#jL4uk0%w#I6SugLpA)=`+m8DgISg+b@PU($R zK^`_=W8Oh_P+dLeqLhUso*N0C#KQ~i+&edyXg=unvAfc*4PsG^m{T8S81I{e^2YF` zX&O@7zss$cIAV1)*Qh7~hQSkXOs8pn>E_dw8W=ZuUZXhF)i;JZePOM^iTm-8FYvxY zV>#(*4ziEdU&;nTY3~ktaW7wXBgAFrmkDp4i|a1Sj|O|IEue92_*2!iS;X@TgZ@V} zlL8GBKX|d_xtnS_T~@UpyvQRQ;M@r!E% zrwbgFha@pMmf6)L+a;~BXUL>*T8O>Jbs+%AeA z_BS9lp?L)pqLlPwWS6gFR5X3|I^*B7-mNHSw>0pI9NA28|L~JEZR*!dwY-5Mt7GUn zVun8HGn6Tp2p0sR_mc%bOuf+1qO4P+_34ps=`~WUN@%~*Ed;YC?Of7Vzx8FRrBFV{ zwaydrqC!^N!3us|=*QurG`t#@vUCZc8wO>;&RmydS3Lez^9#t3VI5+G*MNFrp^z}2 zwaz2LC$ncFG#$vFE9ldb4s%!@>d(?Fn`(}-f`7{PI5dOSRmvaL#h1KpC6<@JsfSOW znu^jsntUO*JX{vV=m9+ui^);Birz}=6@6AMeyj@((l{!Ad=F+XeXm{RPQhdT-mRZC zB6jQi=KtEEK&<_gFrnma=y=wyw{GG*RON9Dt+a$U+iydEsl&a6>^onee;m8%%5d3E zqKFSv;0+X745lk(K%bQ`hJI*U2kq&8=*y7T3YPCyw4X6u9I4pnY!le{9?W5#u!2V? zVNCql_W#->5i7)~>xPVh!R4wwXnjam;3oqOzxW^Jiw6_SH&2B14NtWi=4n2uBVtp} zzu|sxqiEJ85V|D2H|cuU_s}M!Z(%C(7R@TOTI@1bQc`-9Z~pxM`cl>}&d?D}Sxc*R zB1d;;p-)b5eDwGT2o04LBXJ}3VbpRV(0Fk=&ABH2x&cPPy%?@L`4QNbG-z3%#T2*Nd8{aHZOSpxLjHo$I^kuQsrRT99 z6IU)L`(ww}Kj77JSt`&+c-^`+EVYqzO3HP;O4+b%NQLv30&k40 zWiMmk{GiQ}xv4Ak+!GIOo|=a?jL?i`_H+?+wezoMW?>GWx^y!^Z&EMC~q6Wvm#BMmSGMVBaPAu;?*hTBmu^}}yIJ3+ z^w#uKntD2c*k>>1?`oK0lbq;Ac7<~~EUz(pjVe8L(_|LMjw0fnbw!_Z(b8-DpFcxS z=nJUm3kXDl^n!UGyHvne{;e|+7}MTc(IAC$UkooG_?1@foQtr3pUhf70mUx={VS7O z8JPSz;jF$6dK#&`JpbC09HMWi3!I=5yFPR0W#o1iV)ZWIdFzv=vrO#_e#;ujH4FHc zx;&-vKN12Zq4(|X zROIPGG4d~CDUNa49M$H_-Zzq~ix0+|by5g;A}UnVSpMhY6n*7vY7KJ27g7B_LXz`m z$nf`ze%wV=?of9G<9WZvuiSb@lof>iKi>p30{u8x8Xy)w0U+cRh%BszzNj5VR@xS3 zFUdF!ZRa|tpsUru>F{%G5;LuF$OcQ47YsmO=jX%a?ZjcC&&?O|A|$m>|C|$Uj{et@ zXrwj9KYgW2>9_3974M?L@o!8rX2prcIfsZ7Xp#nLtE4rO($j+s`HkCYWL)g*xcl0` z!oY)(3+|NWz(#{L7hL4Cu3gy)PJ=$SoGnc=8y_SZIn~q?Zed{YE9oGlpxQ#2Zs?AH z)Y)Gs=|2zlVl9C#qf3KDeTzEodvj@E&F)UiaFjlokl0Fuwwlk$p&7162E_{qL{a{H zxWvDq2r%XSgBZ{hQlQ1@aBnO5@`ZExrakFaqm|ljTSVB&c=Pq4V$dSeLw*7*)9e#S zNrqwDPY2^%DsL;345K>Oe5In5bnQ+G(kL?~xLS9=m)D z&(?WJVavV}y=o4u)4?X$w{;gj;T~3kzbnw}>>HZh6_%hN07Y~vZou>;#KMviT?G!oCc-d}+pIS;HT5YvBu_HI{3Dh1k2!E&28y3dfUmNqxIK2B zfMHd!K0sIa=*I-tAOWLn0C948IW+W}F0T?;ej`Jlu|NR4jkqhmItZvm?Ydq(W4Dv# zps-;gZ3xMX)j5k`kqrZ`^UPyD|6W85w6S&Gs0yjM zO>2=})`fX}|sV3Sf|mTFMB|Zu85*jmjE+F!jq-?h}-d&L}T{5}Od=<-WULBaDyD4b~v!v(+!~*r0)Oz5Mr3z3_6Goz6Yk z)?v-nklHSE+KrDo{ zwU?JH0(xnQ$pa7d$D`^?|v20i=}Zas32D#Ma;Kz<(I5m#vK+Uv!6 zmX*+|Kcl}9x^^rnEZ{xLI-S(S&n8$WICIxDjhf;pOdGHH`2=X8)hJL?zw-@Xd-xC- z(48K)qE<;o^DX6lmxZqqrJ)^91XZmNt|GUbF>8_^sHiATc%DA17PSM9P0x&TYvIWFggZr66;TnE-F@{ocikbP|}$3Ac~t2Ul+hhYU*7 z$Lo58l`B5{o-&<>A?_44a)sf%t5EbeRe>$#tuXeH`&;HcmtKU%Sf;54TQPp9+E@zS zWv(D*OMQK5XAsAgFC=08d%u4i+0Al=Z}&GOzHj3v@rix?P&Oik_*t%#)x`3Hxk_=b z0?*zpJ^tzr2+cSL|CW!~uYCH?s1(^~W&mUHo!TSlBPyHwhJ1i zW|uH+L2LCG9*a`5K6X33tE8zh9g7KY<+vMd!NNng?gE`JD5mFqMfK{R&2@huAekpp zdAJ4%dcVlMg$^CGnl`Wn3&3sq-QkzN?a*PNb6GJ7jK+0xHojUJC?ss=CByf-LRW-WKGH3|4YYOk4Q8lnq3 z-6CgJ7`br(7lkQL#uN%qB%yEt>LIV#um}$ero}qHcUfyGyu|1$Xa1O@QvBLgCI9CL zmuwGBJUTy_Dwc+cEi<|w1?kj06^cp@QLf-UOStk&RWbZY)1kO3lI~?tp6aa5JjQu` z1-upJdo`imTA{ax*=nKa9Qev}j3nn5$>m-EvXo?buP0}CBU_1nuKoHo6o(BdO9Y|H z6j5r2Yj4UlafFxPi% zC(tGm+e7vYQl3UO@ufVKi;LW$-W+D&g)0;qE(wx@{!onX)w<&YOem!{+@v>4C9cUU zdEgqPKfva9#%=fiSn| zSW)k}L5+Rh1(v=h(j7CXd=-#B{?_-=re*$7hDVp=)OY7en4KCDx~fxK#G+KAyd4*n zu>89HMvmDV>yY_7Et*{e%#|#h<}r*V1-a0=W8Q!LvY7c>Mt=+H_k-BvJS*fRB(#(K zLZveg#Y3RVu%shzQ(2u4wDku4)x{z3x_?8ABTIJ&Mfp#q;y9T4>&~De*XXUgFYMf^ z>>RYWN0lzphHMG3XOAk3&Ce4qH`v^E-~83Hjh4}HX9w$lK z1ycLd)hf$ogi9|%WL{kSl$F>Aa4f%)v1UL1klzK7d*_v+#E)2j`|%H$uPdtYlo$Ju zjhXpFW@dM&u7&a8bB&EFrcVzKQP031LUShL5YXW1By?x5U3O)0N$0Xy#b;5gFF=u0d|o5ls(hy4P^z+yrR#sCPuH5`hE1Py5Eh9uN$a@_Wfr#l@m;|e4}Hm zqtfF3X8V1R;|rVeb|#ovUVq^I)29rWO#C+at2yrvfY&Rvkq2=Ln17_$?T#E#-WEF|H}@ws~){ zV(Fi>571A*^!lA^B;L=v^ZGFweFQz=YD^F4AF#RFL>sSh6;{h!5Dknms zU9`aWFKqY8im=nJZB^3=ay&^VM+GyWJrsz$#vUY>^SD%x&!&ATHm zD<)`oMI+8;5Cq1_rrFeym`9-uN#jhCY? z3b{JM<}VEt#0q=3%gA7Jq{hJbrtld83}54;qG5Xt7&XFCb+G=G%4M-Okl?#gyxVra zhGmIwWRO^h)>^NI6_HO6>Ht^%YD|%p+IxIs3w^CD8tKn=x3+aMrsoy3WuI1HD50N7 zT^^RpWJ5St0zeyqktAA5z%Y^!HjytWBxVB&o436k7S`i6S!lL|H%zS{=ENzEGIw9dYzY zh|uji2LoWX9R(h}sZIkIu*iyTu|P?L#p_M1xkl#2PRq<_Z=}M&SgC(^U9W~ArCO%j z>(oCt>soWpd3?28jo226FqqFIlNZNl!+@(^>007Qj^jUAWHtIxgZJ7citVT83B;sp zU+X#mNct|;5=i7lqwf2|4x=q${W*8a7Qz(T12p^Ezl!CU+RQyNvKN88i<)>qf#YZX z!VjzYDJTmkS)|v-2*?}pv;-Y>PC%>4bO&{G6fZcOVW*k6@Bzwvz_y48a&@y+KZ)jx zVUZRNTh6>noB^Qx( zL;mjd717*+Gq+lyYUSc*5`1~ak_V@QPXPYcW{6-Cj%if7klvy1;bqzz?aYer-lLOU zGxydn1(X!W|8~9rBw4s*4K_<@y#eo#dUIP%%&&z4rGm24Gd;?s4@i{8{$4|%Hu-M7 z8D>hRJfn(#q68k+Z11`PF#L{82!_)D3A@PM9P%_Ua;awt4LjB$WM{r7m;)i%Y+f{& zT{@o*(2DT>V%`uUTU(giKYpFQ?h(S(4XaZOOa#^yw4<=kc%BFFrV=l?$CI!_;6(hg zgVW?ZA~7y7G-r9BK$@S%Ic*cb>!4;G%#&0SJ-_{8Vm+j)0weD;ahY_|1G>*%oG2Z| z4%hIVgnx~Q5X{h!*17~yIYFQ!s2sWLkH98oyooJ9^%@Oe3xXjGd$wop#Mi9GTT&&Twb8xrN&Yl2nSo|wMs$O($xwsq> z5=smHD7N)^?L3YW`O^z02l6>*1Bb^;EipxE#7o!6=ZySpnx5KA{8JC^-+>5K!D&qT zcYa=4P2KNPP;QU##T zF7f1@gYj>dW!lruuGD0my@g8Mo49@fwTsD_sm=Rv>g1b9?cF~-i=S5Y4#j5B?L9`7 z!X>qzzWBbP(j{d;GXj|XQm2S;9U}EIl?DEMB6#a9s9SHebr58<%S(4(Jybi^Qa?p1 zw$vYa^Bww0ge0|&>4BqeA$NQG^OV>+xTplRYGkuKXxIIl0;@(p=Ar*F`ZFX#A3n)x1pUpe_TfLsTNLQcX@s?<|ms-H6Iq|O4A0Kr0vbLWLIoqBy zsOyL!fM&7)yGcp|Q}ts$Bik__CaYA+Z%XWNt(#?6Y$No-kjAsyx<^UHsbU1o2lw65 zGw}>bcnuQ;XL2Pfdl_|f5GHi`QvASR9t#a1-tC8$SO0{I8`71}R>%FR-}P0jRPNg^ zA}%doksCM!(-83(5;Jk}Fr4dUJxu!uGvwIvM#Wq%p1@HdB;N%Cw7q{gk4GBzwX4)_ z0p^;aqnuvK$qdilI!{V!Kk#@?ea2I$0XrG07wz>^tC(%!9leO* zn_u$)i}b3e#6IvK^3OcLBIgRI4;;kRZ+gkhW&G_m5LWu>ITi>&*iCAv=YBAuy{j?W zR~`Qf<=4N`wl%FBPS>jWmy3P0Y+GWu2TpH|8w{2^<=J}%fxhK7>70x)%j(GYuNem~ zqrkksbIRHT&EQperX1+FH1sUyv$^OFw~|-((2m7%|Ftbz&vM|PHU6koN5k;iLJr!a1)}RJA(VG|zv3?!J|BLYrhZ45Oo1Iz^du zKcLJEtZ})+%1CzA(H=nT`5;hTH^3k&@kT`P!kXKYZqL_We7?VRz8gY?8d{XI z>%QVbq`xBsd-ujebZwdo#zVzV7J5GxoWak5JP%B4{Ejs0gN1Bt^Qa^buj+jEw5r^d z5R^?2ax;#HnMo6A;N~P7a)C z=oZJk?ahKS`fqMLdDQxbFLeOvFu5NT8+*XK7!yC3V0cm34f26=%-YZCzfKysp`3;g z>D!(FIt@GV-0XksC=e8)%Vz;{G+ysWkqnM}d}FAx#Usl6t62m^9}E5zZpR!3r|C_xK(z>34|L6ZTP#-=^2rKng_A<14c837cY;4;MI z>wm<`OMDT15-+%$^j}iCd%YTY;`J0(sZro|G56cUp*U~N+kHVLxsQ2%6P97}1~?`D z&CgPIzo1jQZuO|_!?e6|CmU0zF)#Fdn`dHvxYVr^Un@?_ST^+SoyCGIJL@B1ny~-) zsuaAv!y}{%h3Prh`*9GcDQptw&Z0~Fa1}0SYOfUksfsXjYwcjXToVLzWBwqnkw}}qk z{%NOLTJC@8D9mT=NLB&pYV2Gi;f20zkGZ!|Fu==3{v}caLJ%S7xSD8RPCL2LCcM&~ zRHI>4@s=_XIlVMrdBe`)$eY%MTlxNp#Re_(XE8pk{kSFE9x_}O1DimSTIonZ*EY=q zYuy~h(H9!W4*7Simj0oIQE1(r0yh)_u0EXmpDc%Ox^o0pmZNyx>dVqJc>|8OslQqn zYmaKui8~1P^xq5D6;^zAFUd&@f)C(wT~KMmLi2(7w#!2$oY)_Qo%e=dF6O4FP=Xvk z%^9Rezj4s!ag;)t=#3pqdgF1Ty(%HT8P#b&%0}4`BAWkwB)0o}z-fayg9G^{Ha}K% zm#*eDmgy3}*xDW}yPI(&+OF4deRU;s@9X)V|CeN56-Cl4IIb>oT^)A6a^(tnBi5^a zaALp*mf?_*7=cwn^pw8u(ta!t3Hj}nQDNszWAXv;;X$_Ip1!iWvDH8qw%YeR5k^FP z&zKD6sC0zrq|A2C8vLRN-;DjYLlDcILBm>g6iA+e(NmS{O)QX_+J>N{Gs5^MW8vC6 z9Uuj88Lv+^LXgVM7Hc>Qhil&!%vhGrF*p%!k>Kq!iAurvx_u|G^!U+)5L=l1MYWU zLGdoQZ>1xmn_7qY9lL{8JX3+=oTJDED<#RPceH0FcexT)%yAiC^z%o@HBXjn2iM(g zoE&ILX+Ge}HC7tjE+m&0jy zS#;qtjlE(~u|(tmYB+qJ%~Q~!)C*VA^R^C_s^ z*$5GJgtd}2=VUEhLn}i!AVUFVDnbPn^qwjhDxdb8neo($WLs56_%l#(4hBd3Hy#tTR8e0KjEj>x(d(R0Z%Z_Z< z%^+*YDx!5gS#()7n&p~BmdJ{Y(-YbqPu4t}8QHj=iz8oEb(3?({p}wA=dL9HbxbVl zF(kIf$&40&xkk7RW17w14S5_NT8o%S3TF7S@j1f%iBnKf=>zWuIF+x1=~pgWFCwln z8-A$S+jEYyp8iJmp$EL9gM*E>o8o!v!12sr<1ndhPBQpnXh>yBSI6W@N3w>ae%rP?)TBRbQMyyy{4&<6vj1V~oSABlj@KKc znQ1}UL!lvNT4c#8P6lV@3)?7!>W?$W*E*|1-y*CNuC3^0UX@3z*4(^tQ691T>MiX> zPQ!0NQz46wSp~mv3%u11I{U_}H(>f|Z?-mdd^pnAYcjs$e!DPHf9~ih9R~lIkp;;k zDZP-TxQqE#qe5zi;Cg)tjHm`eT|~FDGN#5JNAW$lG?xQfTIm~o+2m-hi#K)N3*f zAo=?|S)|1shwb_rdnzHWO`2Qp=D=plBOfl3V`$0*yu{atUP>1rvgPDp_UnqdPZF~X zn>r8AOA^|nmG`xQC_geky>L=2Gm_zGg403!PNZ?^gZ*^n<7JDL@X`vh=$u>|U z?6zLA7W3L`YQ_nXw7lW#y7yp?xbS%thGv=lA#5gl+2VC4-$aU}7nu7n`m=0v84#8! zNzf<@AJy}X!&`esr_vJKuQ!{mpz*SBJ;7e5Xq=m$H_gj|z`qPW_Yfwg&xuD2gE3z7 zOJM4wE{!l;7plh~@WmF+^r^w~jnh`UB08KKaUe7S-i__#8;_&iB(-E1kqhwz=gM=2 zeXv_C`{ir(%2`fPZ}$m8f9Zzde!7x}J}_!HT2(JQ|Lsz04tHWfj<);m)EG){b??OE zZ1{w){Ubob7|U1#&_g4vjjiv3Gfrb-4$M-9yLr>~$K&W?Ky2bjub8_jzkC?k-WQuH zxrs?+zgqtkqioKsy_$;8I<=%&z0}quGbv}v+jodRup;KaY|4$3wmA7-f7+=09{PFTx%lG-=^M(5I5+1dg>~ zP$nUON*+_MRHy_T?;AFvRIyTqjoNm}eDaeUMABmcTWb^AUxZ8UR7a%p;K2iPAONOo zDp$ZLW9cFLME2bzuyGOGA(>(jF|za-x(87mFMwrxL9`07qrvGa|HYP(!}BB{!@2Up zovzk#GjD8KfrCoTDUZL3y!GJkgH4_^hjHYo)i9U#cAh8IlpavioO|EqSQU@^A^O`m zotbcVnZG=EbrziQ3az^+4}^!`f1PdD9Dvcur}I`JxMWvkuh>-JysT!e+sxT$5T&^d zf<1WeJkd7w(W?KL%&S;l>&XVyo4ytmXv2<4`z}mQxfURctgCmv!d%bBLxOL%B^vSW zdnRrSsX4Je)qd2yU&aHfTeBC5#sN#c)F)up%gQM}O47SRTR9mao$A?wbE&jt@FMN4 z(YF=z=^5a^aN5mx161uv-&Ast zId8ZJ)8`Gyc5NbqM{U{!8>#oD*%$Sbsv{EtiMIevIc4A{z~4Abn|-V|M_9N;uD`2& zIInWu;K%AXGQ=730y=g1bd1>}ujM<6;tDK_VW>-;ls~} zIj10=fviD_sPsx1Oh7Wa$R(oCm~O_+`_#hSxKEJGUWP0sjw(b3W0Xo7@4wvzv|Q zw6;2cq1wnFQ-jd*vB;$xMgu=g2Yg3x2;Mbgh5-yR=NIN!BZoCC+!9rBP5J)6?yV`_ zh}+KaC%?OqL}>!An!~!(pan+m^6v18dz#VIdCmA`!1txW4WJy5?0TDH)cS^5W&dzz zG?w|%+V?teU^D=_W3;!2uM$uIO{?2P6uO#UB!AGG@` zg*&)FI`BRuFtty-$D$TG05heLtKZ1o_mEDm>hk5wNqRvuT7P|CKfCBK>SJ&zKjjIJ zQ^_q}U~W57(X{IUmBg&(DzK{bIn+1KH8BhD-G-MERaX_{5p^NIc8nIOC7j5+abL|B zeO)oOHd-4BXdq4bBmRTpswygJ32As=ChR`|_j(q|7)4FvC$?YrD&?^lAe7!q4+h`J zVx(M6PVwZq=uU*;*Ccdna2LiNi|Nj&-~`z5q-$4@$MD>omHlQo>1 z;6sK4VyW)WCGp7g2zd^2aBw6E@3{B$_SWNT<6MmQdUiE_BlzdGqdVp6eZ?WuOOXh{ zXk%jk^JDSJKn^30bsBxWy}`s@F` zM+;U3h=$ST#=UzkkBA;8D22atO^yF8oBYaIm16pA3Jpw)-&W~8GlLpz=ziZ1hrJ_8 z{vfV|Bl)YZH^53f6FR@TjhRbN%i{dYX-Ek;7$>@JPq-%PfH&@Y*B$waJE!^y;$|dH zyo$05t5i=m=G;^_t$c`YVDRf<>FD3F^&>uciRL+53LgPtfeXIORKhagYA~rvHQ6?Bb^Pe>>Z#qEsku zthDP7X7M;=G0@>xhBkC^D666g6e%0^=YGHU#<#)ROl``>;*fzWsh{N$Eve{4fq4DK zRPf&5&RCa|y|)*+wxF`zSa_Y{V6rl(_1AAnT=AcG3E30G@T(HMsW+E&k`GWm?U4Z5b{h7+bLVHP3M-wqb+WEbgOt!RW0l_!*+99@UBX>9i2n!{XW=?+==NeT1^O@3p*@m%CpIBDEmrpSb ziaQ8xtg0M2Q|_Ymn#p`c|4mcO+t-@dhrgVu|6Oya@X<2?j+;sxMm#yTF-nv41~Rf?XbQ?stTE z`iH14T@;h~+27xO$~7q`T$+%_`{aAfPgUMWG+f4@5czAdllZjWf}8)qYe(uMPt+89U* zrS+0jO9b`^c@}>LC_ndD&CXSmgG|Se)Bb(V7VBURUGQI@n+bo*jc8A*HvA!e@n2hm zkFIukjsDENqq$Jpr0QNZ8Fb>H-=_2DgZtm<-OISCifG)|r!)QgC<@0yX}_fRYUY!< z$p6;%NP&^SF}^2lijJw_fOnoFi0QWq`+3OdvgTFdH3Nh^WdSs@x-}#}ZA8(-1cpT> zd#qOCr|{RaGnALp?x~uoM^EU}#hko!F;bK&rEI0r1u8%Go{P7XoL^t)>+Y7a_g|qT zL<2EP;t!g2?8U^=4<-PTbFus zrcYyZ$n4*A-eSk#@N!Me3Fjsc+) zI_vdmi~2Us6AX&VpP^Y<`)hsuvdk1DmUA`h!-w;2w=Lxn4bia6GLl5J-X{mj&SW@q zL8vB*nC!E271$TQnM?jmi{_lvQl3A?vfcE{8^I!V)Z7*Ly}KlYhgj4a*_<>MloLK- zyArzlZ}&xdy1P?vzL%}FvpPPmxG`}|HZpOH-4>8;17oJH&=!|?{eIuY^*5w?9^Vby z`S1JLIzror>TU7$_Z_K1yyHKMmd8(f%$xrOjBX-=~I>)aHXXWYd~W85kCnn<543fowm2ysK26=@R4w zTC#6v6suhO*W=M&q9^a#ve1M?KonPRI~4j|kF>#CN1tTvk3G~phY+u&vnOnXSeesR ziiA&xmR}%6_0QLNGHnJ#u{eq88c8Iy1v^gAVtDm;RH;gm|CWq%i1UMV!$h$TgB#w} zt)7*ir@1d#-{QSRxX|_6>}rDWg>4e(i66)sUrNd0Oq8pxD^V3=i9o>(c0oxbvY=icuf5Pcpisk<^#0WLc5#WC8XR4bedz6shdsya$wPA+7h znM=;;dYhf)bd15LSUX7b&mUTM*&C}{Qu(rJ6gJ2~0qG!QfgH62gp67*M!SrC_F8>L z%*4CH7$J|Cdz4AL6SSqN_>U;Zk79~gQlYY$cR-CVl5N2iRV}drzgyqBhf|!Ajw;;q_k z0k)3g-lj?3GXxZ0WAhHR!x|x$!lt~r2{{Vv)|QB&p#&)j-85f1XkE|Te%8Epte5ytxZEfR0OXeF# zTuFJ}Nj{Tf>8}Nd?BIu2M|MG+tqBKeU;x#F?jLM!`wm0tBlBXI@KN!2=i2^Cl{PUl zqT7V96OaZRYN1R;pl3n!CaClIl%NZMq~|vwz1P>*w?N3e|3NX$9;h{dEA%p}Ma1qC zG%_vb6E3_HBLqLqk(LO9zZ7Xj*c8rLZ!n$3)IgSvceaB|--CifybZ-wlbbJ@6;Yy& zw5b6#ky2PtV_Wp6cE9lxl$^S0XA}WRvSt*hmlS>M1>fCV+o$azc%!52WUs3wwC*Fm z5PZ0g=r91_=GR_ddVqM;ldg0fN%;v%&_u`yALhT&DtG*(rhrY(^*%LK?2J053#3-R zpr-a2Bs1(Ik-(P>&D$f0k`m$YXVtP`!^;!@OWs7Sb%V{aN+cMNtdlR*k$Kj8z>~vb zXGX)NV)gxa`lBzhrW~E*9P#$YqLC|UkUT1zFt;jSlbF5AFfuiLBz7G{l#plLyBa56 z%C|a{9ci9|3B88|FhYp8;S3NE#;mVfACi9m;)6aBNGUbBIobHaNBRhG=v)L~`tOb4 zCGEFy8|MJk7AU@DIr;zb)WRRLDPi<+c4EhgMb@w51`1u#fUnO{?|1%0apoB+N#(ULVH^^!1>fuFHBC$ODQ4N{DwsV z=oqPVb1^Y70jw9`)^aFFg_Nz9H(a8T>8muo)uFs%PF3!QkqoTr`R{Xc6>|3h_`=j` z`hPz}WQznydL*?mkR!Q_0K-Aq<)e3tLnV;?8o${IG)V`jF^f5~5ePMhQtbZfPLMU6 z)}foPBhfAg@M7yOq`Qz@Dq!uQk~H}J-y-ulWsGFU2o(JP`qEXVA9aV(9EA4j7^yjZ zA2s>_xz$Faz+Y1QXGzS@tF`hBL$dB?A|v9;#}PnB z6pwtEZ!8b5L=wI#T5W}Q1R%q!swcUi0_mG_ttD_|DB`0MCGidcI=dK;m&jiz^X5j! zJts)8O&Z*`u&~hiK>X})DMX<{ttu7!~e{_*{Riu3}M~w=79KFdZjjBk-jZ z2*>cH)I{Qq#LQ;}Fx;PbOk;w+!{Co}*9`_hif4~5-g%xmx)KVT4v#XCO+=DVkqO1< zqrR#mZm!{=YD>I7TJg_nOrH3Oxvb~8*-UU92`-{pq53cJ-B=iq4!8zG-nzJ zbnYZTFnf@GX{>8`Bty6jRd|r_m_{O_s2orsj1E96+}JOuTo*uMDc&%-?+=bPZ9+Zv zMz|zZH~}UJJ63B?$S7bf@J}-3<0+9)p8P`Nao*=TUYPn)ofd@mtE0!^Y4d&=wEgo`NsetA0l{FAZ+f{=lVi%U3Zu%IZecuoa zwAs{{KzELkJ9|FMfUI!xaxg891wLR}oXrI&Bwy;% znEWbaPPM`Od(FJ&ciVf&u20kOF5DN^oMC4u()?7rZJy`3 z>BhSHw$S+ueE@8F8MG7`GopUsV0)(^ssX@1{XOWN(PH&vbxdSeW~%6_Vtx+5*&ZMW zm5_%8Ex9+f?j)*Uk_Y_{f&x@jG^S?k^={d z`w=EP9|xjp}rMBFpp*`Vk2w;CPdVnMj^Od>Ik4zU@p_Y(EdLq zCnzyTTuJE_>j9|PS+$lfBtc{GtvmAoa7Izv&@ho~=j(OmyF8nrotihWhULY0-%|^@ zs^SY2YXr!K8|0fohRQ(A5jN8Bw9OeH8(1-1`C45GUV*`PKWqwbf(h)e1YFq!`8jWY zv}s*Q_>V(ws86{MSz7>F!8PL$pc{we7`33YZOV#@5R`wyiMj(Jx`q><$8tzjP0i^m zrNdJr!U*%BgvTO3ceb#56|NSP>+?cN5Z)^1@LGAgO=g3u_}K4KBTJsp-fW6vP&LqK zUX)bpBlU+zNtWXQq^(l!z5Mp73wel-?(jMu{8+ob21S-(jPYxen|fh_hSM<=t6kt| zSVy1zS{)DNl{GSEPe*bbZ|hnOmvTGyA0+>WJ3$dv4LVexh>82nf&8HlI_~@rV2`MX z=6w%rk@^Wk-)jUu?gh+Dlhh923PqK>JaRXfIC_c+J_K7<4pdcls(z=T0@nH<*mYi@ zxImU#+93yrsF3yXw)nARaCu;LZT%g;b}K_S_rhR?WZmwvAt)OmFg_ZIRcs6K^3@JG_e*eUo zU@z31386HiLRGJ12wW4mN80xN2H;cTe$?FLya|mT9zA|U@Dn34F;Vlauo&ctc7$vN zIwOz+%Sx|f5ZOC9IX(7yQm)In3faZY%%=rZ6`+g`v(*g8t9?J^;m-GQ4HKwWef`u? z_T+nQ;C1P+f}NeJW5F9NAFH(rOqE!l*uMO%@C17Y9N%~$x0^QvpH{b${S*9vQ>DKm zPYO<0rn`5>g~NmWrdKRmY~A2zpc022&#P3WLlM_uo|%~$EdL0}i-P_MHU79Ooo}0w z;5Aq$$XHqp6|aE-a?bPlHlU)+XMMPi#m+JSD(@Ju{(Weo(ISWBiacXpAw`fM{xS2;>53fx3r0C1UR~z-sY(bqFhkZS##LipSySTPR9G6z>^6gwf%5Ttlf5x6f&aNV#h^Z?Vvg6s zJYF;Bt}`(+Bbln3nB?9X2DWwv%8sRibI?{r5HhEhkPj7xs)<~QjuxKMdAHRs=ynte zN=iMQXVntw2w39pykKZ_-T}TI_--4}0OnH92r!EB7$~4?Rb~b; z>KK-KbO*igMc?jCO@%=LVPv6C%wojAdNm}r&0?C())PFA`XBbTQ;J78;K{GEre$M@ z#wJY25EmZB0DB%sw!0#$4w6Ba6Q{U6?}5-}Q{5GJ?*f;jc@rYf#(p5E%q~wq!Ql7* zWr_%98Bn99_q`^tt)9h%XoM_Z zwnnC2TaOkGz-eSzC;<=VR@?kezNYisTjIGkWi?i&~B&1NoWZ#TQpK7qX= zBlG&Ii13XYP&*Fw9gvHy04bQZQo~(XYhuZ6pdN#EO{t?XN3o*^)6fzJb4lis(Ujdj zGK3lRF-F+~5-k2sM7Om_Njl`KKC0!t>o`MQM-l2$0lw7gGc@p+@&uYKMk@um4ndyr zzglmIc1>VwR6;u~-?7@gy*=>vOtZm(ex2RQ2i#_e&u;SbEilbR<2Gso{s&l=u{{Y){!P>>cHiMVBe}ey zx0f;y$b_Q_n|b4M9?2R*#tvi5EwCm!m=uud`mg7L7ZJ$qin+Eq*X2X;abW>CNLC*G z22phs@itMcYLD38|73+kGe7aK7P6iUz-iU@LBsAe$j`a0y*n)rNkJ~J{~ds4gn3it zrUPrsv=~@0K%~NMje{;8pb&w-vg|}QbV&?z&~+y}mv|e}mmG=`p%+a;G0k54pc(7> z3pq~y``HK)U4AjbKhAW&J7HGo{FO%I!)`s}(I$_>1?}rkAnS+Jk4uZ=3W%+(txu_E z9PPk#17qt8^Cd$2PXPY_uKu5%Si^S8>SmYRf@kPSWilUWdMK$Joyv?M!9&_<_5KjbN}bFtyh_(rVD{b z1leEQHmQpg8M(RFe^lU_LIMvNc?K^Im#HwIg@xEbUZYIQ>FyWa45^d}9fDn{hy8tY$+J43J(q5Vl_q%N0V@XA+V7ApmZfgS5t@iPOO8I`JzG`mu-lyl|2Kin!mSfLJo%A7kXRq{; zUPI9Xj4s!<{ums=x$)T8xI7YENMrYFFHJfWLXea8?H8-#5AjjKdhA~E zP>|x88D-ni@@7}kp3P?>o3AU}S?_b(6NA_mo8prl`AeP#-Gd!%5o$JOvGV5mz#s3& z&-rfd9>L_i-+?Qn-$2k=Ri^Gt)a_6Vwe@`~KStWZFSdUFRc;P7jbFb3!m-hD1C{tw zto466TyvQ9QjiEFTx+s{RU&DwjqCm4q_Kr!lRj3(&FirEPhyI9=s*(Y@L{c#DQo9ivOD3Vzt0m0=@$_;~xCohvk=cb;tMZ{98^mfc^QI0_gxw(#~D zJ@isYR{X6K8=8!M>pWB6CGsB|UxF?-YsdCV-;IB@J*6^IhNhL=*c^imBd-cm{8G`R z4VUQTZE67%-*ef-l$6T|`A4DK8W+o5nFOeeAt<22Gh%o4PXRkf38QPaHOZU?a)JzT1+y`fE((R9gYruV4!xhtw#sF)fWrU`m2mM^p~Z|HAP ziL9a8qG8vEDigJ%y~-|bCN#fR4rCzO9`}+eUl`qvuT*x?)YNn{!EE4F;jpdM*wLKR z7niD}S~j%Nn`CxkAg&%w9e(3}K@=7H?_FOKJ+reYN>cFArIE`Nv-e#7zt7Shj{N-m z?0)$$HtNtDDFxO9U8UbU)t3HE9QymoU5qi1|Gf&Z7MI*`Wob`&$ZJ80vvfNMbbWwx z!JZ2V&XH0l@rDG2@<)%%7N>{$SUV5Y9s}A0g7EvRT7PrWot@yoln1|rTal1QzyBcs z8KKH>-DD6+M}ux{DxTNyHbzYcl5FWsn?^MuY2lhlJ#|J`YM&%d55h+0C8+hwUKaGm zD#7dE*gde&-$?;*sLBx!^bD|EceCUIWVd^_e_l!exOPPB1P1&K*P$0OyoIMf7)&l0)C}d zR|r7>M~7JBPKs5n!uKhLB36J5r;(6k9u&t*6D40Z_x}1$*Hpa~ z)carI?tUb#b$$Vh^nGL0jfNv#ep;9dw=qVcpgy{ti6*#-Nlfd4_-!8 z@_npRZpOwvP>Ephg0dSG8~rYQ4v!x?>!dfLoDjdb1HXcV`RPEQ`B_Ok-9>FUS_Hg` zt?Gnpk?lwxF>t^h{rljg1PvW@iV>jz>7;*|Zr>l=$EHBR2yS~wxUAjX7a<@0K9i~d z3FAmQnC}4k8f9XGWD+h`@%)Zrm+>ViX1tx{qNAPD zSUV&sUXIrRwsv$V%JQM9slN$fVr>%)*RT&>P^&;tHBm!-{tiT z&0NmPWv^CzhKK*R3D`vsp4zo?>9JE$_ovCy17S7Uue7=#y^r)P|L8+AnBF!HmQd$SPYt6Tff)*66<4*> z0H$>4F_JeJ46XQ zo!e716lrL&^hhNTXvu;nPo6*u$l}xwg4zQ~mi+*Jz1hY$qnlx1d3_0(NGK*+Aaw0> zoNJ6_SnZog>IYTHHA9U^VLfPpuHh*q$eo!m=cmC(W_fV5Sz$k*7>FxGm%k9vLNDLr zRdXsf>I2MQG#5OVJFj)R^-p?=O#ulU9RDt`9Yo(cZgGh{pEk z`dEXdug^Sb`_W$&=$yd3l87YhjS(J-2Z3Q6qtL8`FnV6O@^m61(Z3stJWQUPS^$UG zp$=}&ACT|qrq)oOZ*&=LtHpINFcnNlgguMkv^y7hmb}j5t}T}J4%qELT3c{PSchsD z0e*I=a{=5PPJOD;UXm@sl(*Vn<2|Qq^~^9u(P+%+GE6-EzCRS%Iy~7M!5=$WU)2ru zKJ+qA7ydc6ef#`)S87K)e{CmX{p<~`0fYArQ*)o=dY{G)&+0sL?Qwpf&wYMX_bW78 zxOl8LStyaHvnDstCa!3wkKYbS7SXbAz zQTp?%JKF}aCt#(4=LO3ef3X>5fg9a055Ut=jY0PwOy*R{EW}y4Gzsguq+Lx z3;sC?qt7r)8S(QLgBP`>qQd5DBIyQTS3?wlK;CRVL4YCVa+WNx#&)l_lN~k?+*bxE zKfceGOX;T#Z+zk)@o#5=k`3R_{hihqhzd%*6J?Hj^#18lyI#gJQT ze!1T4gr#v;;h~`bzY5Dtc5>DTcme>??in^t=J?O<@xXe+v%$6m`8e{OIspWaDZynnoyc#62O1fNIsIfm!l`R6vL z*GnVYOEkPIo$i-R+)wM#o~{o~`OY8E0HZ=%V`t}l=}ElC$lJu*LM~rk%#Sig=wrwH zHhatrW9dt#pt7d!g$q?NLb#q#AyFzgIT!jHkQQ}4M7S=t@+>VgfR7-)0<`}y>5TmB zc_9bI2>@fFuZAW#f@KLY>tAm-AS1xpevY%NZl21`2e~rrKsKN4o0MN8&wM8N2Nc_w z?Q%D={jWigdTltmEWk=*bL{;Ei2U~5vtsyA8q2lk(u~h)@YP*L!3blK? zy7Gzh00wR?mv;c&!}WadYlK@$P&*$AuL@)t6C>b-XPDkpS#76m2errUx8D<@ZkIRq&$9QNg)#DIUo zbBo$NSB^)UXlrNVI>r1Qizr|%+x+0_ozjzf*=AL+I|1UBm3FG=4Mb?PH#}RBQnZ=i z&PGVuGxxfL$}ib3Pr(L%$Wj5ox|e(;s6tH!ky4QYd@E`nth*1==*GvuZP<;OzCA-= z6SKJ6ZO{{+5mTnF4+PyhX-7nIG%F!1JRQhN|FLu>ttuw|vEuUM2Djnk)){OPk~GF0 zp=NgnYHH&(gI2+u|3{b121N_t;kGYk+W`tioQFUx@~ROY%DPS-M+fN;YSxRbrN_j)Z- za+@sjkAXvDG6=2m&qfkDhZn9&Pa}fA=J;%z4!C{Qy}VW+CMW>=HWXrsJ`O|OIH#G^ z3!pB1)nJ~|x%c64AoFL~#45!ht@}q$WDhe!G*zB4mIb;Dzt2?za9?*QK|VLplV-&( zOi6F)FCBgqNcvw+g=9gn9M%=3GjTFZ@ehF5k))Pg*FWJAOhMLsw*UojYcMa@*)2gy z4ByX3Q<_O8zk*x#hi<^1f=^lfIXDa%*d-}vyq9#0jg9SZffxfsTVBA%LzLPG<(xr# z(F?&TxGqkajcx`Lw}`OtJLj}!u!tdqQC4crzZ?D(7Sl6YHwM&vpL;N{-k;yZ9@-0q zL2SiUirrt{+$4q|yI_ImQn=v>x-^)o5HnwiEP}VTxP_--sY?$C2;fojg*{j`<8n>cPVhfEn zW+*DjL?pqB`eH-AZf$dA{-c-~Y-VC!@Esel%F5=EA?7mk_d40YNL}*p=U4%m(Ma7G zkv{jS1=uuzhMt5-W~5Wa|BG&dyoZZ@^jWMOI2;>cuh$LP6&7HQg)h(Ora&nn zs>s8a{}pQn2sowB@N<~~c>tSc8&Y5&o|5av0v1w-MBW?2J?Iz9u=&OY`capZm6+_j|~O{S@Y=w9D(PsfnWf|L0!6*H!W|6 zhK7*U5z=?)JeK#J=8z@9D z+9@r!r6$GfPn7%gJ@3A!R!Kyx6^JAQMzUxW!D#cQZ%m)lUiTk>$k^u|hFZ3X>NgMk ztX@`tGsLWF`i8v0)}9Y3$L2I;yasQ|Epiy^?}>Miq-J+ngFi7Vi5WRzvSpdNZd4q@ zew>}!Tcve0L`OLiJo#~CL^FtF9Rxw?@(m)=q4vG<%lQl2yGn)*}M ziv-#Zo=MD48Cy%{`0cMV-g_jywPZl#&))$>K%DNPv(4NG_HV1ly%dSbSHRQR{W2?* zEY8Tz4Ms<&T3YzK^XPJz4n5&uu&{Vv%ci|1UxY_dtL~T$n>B)`?qjvTjE)uVoNOF) zzBe9}+$7hYRU$TS5zPPh6yvQHR=4(%Mr@`Pj=HVh7XQR*aHZwfguBj#bAI2N)%dR)zR*b4tLtwtbShP_{ba zHV9J_%PuIm7_+Ry?N3X~y5IFs&CTZg(!=eA%E=kVF1!78$o8wIWE|>7wqSv2!a*5X zvbv&ylWWrC%fsmDFq8OegF{|!s(W52`5anTNJC40Ycb9??#)&)@QFPHe1X%~1G~(6 zhj+>DA@M#PJ~nZO6W9t8h^eQT%vS#kou)|t{L8Q3;S#0S zu`A9E?3x^*?Y}OQ_*cNP&TEuo6D4?jdXqQm)Pw!WO%4o{V|Y5XpF=E^ zub8#VYh^o%F!OfVuZO-R)u+OIy_2*j`8ZK-7G++qr{s-L_V+7j+^E^LU`S}&S9_x- zF$#XK%FrmemQt|Olnq<%sxTlk&oI#PQ;p%L|w9L#?@vF+( zt9s-KG84OELG62OSC+oq5LC;%Z&o313%&K$Z(p5BMmZ{73}0fZ_S?IC)1}niC{1x2 zuVYg^!K?~bwz`rC>m=&gq$#TN%+0S+<6{IHr^IrTmD!nk01L>Y!l=HczN;XrT#n0TLu`IZ_k z_3;E(g_(g*^%+;qzJsVkcqPGoNXUCyf9aI!vx02ZpY-&-TdJm?H!1c;yM`}9CCmI7 z?@s=a^D~O{wO^VH*BY|sW@%e%Whar99fiuXUUKHjetBYvcD)y(o13~EaX1v}?V}MY zifVbz`?aM&?ed^=JmC}?IqR%ub9-?gmn@`~PQa0yC;w98Xt%y1lQlrU zda(k#f zehWkFW0?kI;$Zyl4_ul?FR*EAwY9a?+&Ailuea}VSehtyzik|QAY-098*3f&o$4wq zWR9kw)yCSGh{oKr`BPS^t8EiXQQ0$wJD6viS;0*7X7#j;h#N&Ft!kJgxG;licowIm zMS6XRxk$>Gl_x6EFdW#Ewr_l_v7ShrUq{uP9iQ`oFJx{1?c^o;kaPAm6k!dmk;@Dul`d25^NetY4WXRHa?Mx1+~bNF(6eDLL11H|E*CK-wrr$|zH_Q|i%05vRYdBAQOiUths=#rhs3sy54go+%>jTAl zDT)BU*>~@h)xzF3pMwW{ZZEBs@Q66&EgP^pjJTOr4y#oUD4iuQSc(@o7kWVhShvU% zjU_L^qR-zGG@fI+8--GA8~)thjqxYF{t;C9u~tDOhq) ziuXtQ7R74Dj^TqXF3auiga(RfXygG>rO%z7A184#B$N}wVSJ|%-nf~EDLFx-9ygZ% zh}irbAn@}F(rIyVQ#;_1tar=3X21uhw6(X#Ja9OA^yq}&E%=tZwLr3cgDsCq@5@vo zmdCu25^dvae9hz?JRalb;D;Yw#y%Co)%p&!b;?i=y)&_~5za@(e!$ra>}Bu4NqXuw z?dg#b)#{-lI*W^%q#kHgcfJ#1hbt(vY|kE)gA#T d{ZQ%;E`k#F0b$s?7XFD+SJ6_=JbB6Ie*l=SId%X5 literal 0 HcmV?d00001 diff --git a/docs/overview.md b/docs/overview.md new file mode 100644 index 0000000..b9b3eb4 --- /dev/null +++ b/docs/overview.md @@ -0,0 +1,180 @@ +# Pixels-Sink Overview + + +![](./assets/frame.png) + + +Pixels Sink目前可以简单分为多层流水线架构,每层之间通过生产者/消费者模式传递数据。 + +## Entry + +介绍Pixels-Sink启动的大致流程 + +### PixelsSinkApp + +目前启动的主类。 + +通过`-c ${filePath}` 传递一个properties文件,该文件会和`${PIXELS_HOME}`(或者pixels默认配置) 下的`pixels.properties`配置文件组合,构成Pixels Sink的配置。 + +配置类位于`PixelsSinkConfig`,目前支持`Integer`、`Long`、`Short`、`Boolean`类型配置的自动解析。 + +### PixelsSinkProvider + +实现了Pixels的SPI, 作为Pixles-Worker启动(没测过)。 + +## Source + +Source层从数据源拉取数据,并进行解析。 + +### Source Input + +| Source Type | 说明 | 相关配置项 | +| ----------- | ---------------------------------------------------------------------------------- | ----- | +| engine | 使用Debezium Engine,直接接到数据库上读取WAL/binlog等 | | +| kafka | 从kafka读取数据 | | +| storage | 利用pixels-storage读取数据。数据格式为sink.proto序列化之后的二进制文件,格式为keyLen + key + valueLen + value | | + +### Source Output + +在Source层,不会做解析操作,而是直接将二进制数据传递给对应的Provider + +## Provider + + +Provider层 + +```mermaid +classDiagram + direction TB + + class EventProvider~SOURCE_RECORD_T, TARGET_RECORD_T~ { + +run() + +close() + +processLoop() + +convertToTargetRecord() + +recordSerdEvent() + +putRawEvent() + +getRawEvent() + +pollRawEvent() + +putTargetEvent() + +getTargetEvent() + } + + class TableEventProvider~SOURCE_RECORD_T~ { + } + class TableEventEngineProvider~T~ { + } + class TableEventKafkaProvider~T~ { + } + class TableEventStorageProvider~T~ { + } + + class TransactionEventProvider~SOURCE_RECORD_T~ { + } + class TransactionEventEngineProvider~T~ { + } + class TransactionEventKafkaProvider~T~ { + } + class TransactionEventStorageProvider~T~ { + } + + EventProvider <|-- TableEventProvider + EventProvider <|-- TransactionEventProvider + + TableEventProvider <|-- TableEventEngineProvider + TableEventProvider <|-- TableEventKafkaProvider + TableEventProvider <|-- TableEventStorageProvider + + TransactionEventProvider <|-- TransactionEventEngineProvider + TransactionEventProvider <|-- TransactionEventKafkaProvider + TransactionEventProvider <|-- TransactionEventStorageProvider + +``` + +Provider层通过实现`EventProvider` ,将Source_Record_T 转化为Target_Record_T。 + +例如 + +| Provider | Source Type | Target Type | +| ------------------------------- | --------------- | ----------------------------- | +| TableEventEngineProvider | Debezium Struct | RowChangeEvent | +| TableEventKafkaProvider | kafka topic | RowChangeEvent | +| TableEventStorageProvider | Proto 二进制数据 | RowChangeEvent | +| | | | +| TransactionEventEngineProvider | Debezium Struct | SinkProto.TransactionMetadata | +| TransactionEventKafkaProvider | kafka topic | SinkProto.TransactionMetadata | +| TransactionEventStorageProvider | Proto 二进制数据 | SinkProto.TransactionMetadata | + + +## Processor + +Processor从Provider中拉取数据,写到对应的Retina Writer中。 + +TableProcessor被`TableProviderAndProcessorPipelineManager` 创建,通常每个表对应一个Processor。确保同一个表上记录的顺序。 + +TransactionProcessor只有一个实例。 + +## Writer + +Writer需要实现以下接口 + +| 接口 | 说明 | +| ------------------------------------------------------------- | ------- | +| writeRow(RowChangeEvent rowChangeEvent) | 写行变更 | +| writeTrans(SinkProto.TransactionMetadata transactionMetadata) | 提供事务元信息 | +| flush() | 刷数据 | + +### Retina Writer + +RetinaWriter类实现了PixelsSinkWriter的接口。实现了支持事务的数据回放。 + +#### RetinaServiceProxy + +用于和Retina通信的客户端,持有一个RetinaService。 + +#### Context Manager + +单例,持有所有事务的上下文和TableWriterProxy。 + +决定了什么时候开启事务、什么时候刷数据、怎么提交事务。 + +**分桶** +在处理RowChangeEvent时,做如下处理: ^2906a9 +- **Insert** + - 根据AfterData的Key获取对应Bucket,得到对应TableWriter并写入 +- **Delete** + - 根据BeforeData的Key获取对应Bucket,得到对应TableWriter并写入 +- **Update** + - 如果主键未发生变更,则任意取一个Key,得到对应的Bucket和TableWriter并写入。 + - 如果主键发生了变更,则构造对应的Delete和Insert的RowChangeEvent,这两个Event可能由不同Bucket的TableWriter写入,也可能是同一个Bucket,遵守先Delete再Insert的写入次序。 + +![img.png](./assets/TransactionCoordinator.png) + +#### Table Writer + +每个Table Writer持有多个RetinaClient + +有两个实现: +1. SingleTxWriter, 每次写同一个事务。 +2. CrossTxWriter, 每次发送RPC可能是多个事务。 + +TableWriter写入行变更,(对于Stub,是同步的;对于Stream,在Observer回调中处理)更新对应的SinkContext计数器,如果写入失败,则Rollback事务。 + +#### TransactionProxy + +持有一个TransactionService,提供了同步或异步提交事务的接口。 + +异步提交事务时,TransactionProxy线程在后台进行批量提交。 + + + + +### Proto Writer + +用来造storage source。将数据按照顺序序列化成proto格式。元信息(文件路径等)存在ETCD里 + +### CSV Writer + + +### Flink Writer + From b8cc525dd1ec0d12500ebb724dac0973aeee81d6 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 27 Nov 2025 22:19:12 +0800 Subject: [PATCH 30/53] Add FreshnessClient --- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 13 +- .../pixels/sink/config/PixelsSinkConfig.java | 2 +- .../sink/freshness/FreshnessClient.java | 237 ++++++++++++++++++ .../pixels/sink/util/DataTransform.java | 61 +++++ .../pixels/sink/util/MetricsFacade.java | 9 + .../writer/retina/TableCrossTxWriter.java | 21 +- 6 files changed, 338 insertions(+), 5 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 9892a25..5e2d044 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -19,6 +19,7 @@ import io.pixelsdb.pixels.sink.config.CommandLineConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; import io.pixelsdb.pixels.sink.source.SinkSource; import io.pixelsdb.pixels.sink.source.SinkSourceFactory; import io.pixelsdb.pixels.sink.util.MetricsFacade; @@ -41,15 +42,20 @@ public class PixelsSinkApp private static final Logger LOGGER = LoggerFactory.getLogger(PixelsSinkApp.class); private static SinkSource sinkSource; private static HTTPServer prometheusHttpServer; - + private static FreshnessClient freshnessClient; public static void main(String[] args) throws IOException { Runtime.getRuntime().addShutdownHook(new Thread(() -> { + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); TransactionProxy.staticClose(); sinkSource.stopProcessor(); LOGGER.info("Pixels Sink Server shutdown complete"); + if (config.getSinkMonitorFreshnessLevel().equals("embed") && freshnessClient != null) + { + freshnessClient.stop(); + } if (prometheusHttpServer != null) { prometheusHttpServer.close(); @@ -85,6 +91,11 @@ public static void main(String[] args) throws IOException throw new RuntimeException(e); } + if (config.getSinkMonitorFreshnessLevel().equals("embed")) + { + freshnessClient = new FreshnessClient(); + freshnessClient.start(); + } sinkSource.start(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index e254b54..d5f60cb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -170,7 +170,7 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.storage.loop", defaultValue = "false") private boolean sinkStorageLoop; - @ConfigKey(value = "sink.monitor.freshness.level", defaultValue = "row") // row or txn + @ConfigKey(value = "sink.monitor.freshness.level", defaultValue = "row") // row or txn or embed private String sinkMonitorFreshnessLevel; public PixelsSinkConfig(String configFilePath) throws IOException diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java new file mode 100644 index 0000000..c601585 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -0,0 +1,237 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +package io.pixelsdb.pixels.sink.freshness; + +import io.pixelsdb.pixels.sink.util.MetricsFacade; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * FreshnessClient is responsible for monitoring data freshness by periodically + * querying the maximum timestamp from a set of dynamically configured tables via Trino JDBC. + */ +public class FreshnessClient { + private static final Logger LOGGER = LoggerFactory.getLogger(FreshnessClient.class); + + // Configuration parameters (should ideally be loaded from a config file) + private final String trinoJdbcUrl = "jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x"; + private final String trinoUser = "pixels"; + private final String trinoPassword = "password"; + + // Key modification: Use a thread-safe Set to maintain the list of tables to monitor dynamically. + private final Set monitoredTables; + private static final int QUERY_INTERVAL_SECONDS = 1; + + private Connection connection; + private final ScheduledExecutorService scheduler; + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + + public FreshnessClient() { + // Initializes the set with thread safety wrapper + this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); + + // Initializes a single-threaded scheduler for executing freshness queries + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = Executors.defaultThreadFactory().newThread(r); + t.setName("Freshness-Client-Scheduler"); + t.setDaemon(true); + return t; + }); + } + + // ------------------------------------------------------------------------------------------------- + // Connection Management + // ------------------------------------------------------------------------------------------------- + + /** + * Establishes a new JDBC connection to the Trino coordinator. + * @throws SQLException if the connection cannot be established. + */ + private void establishConnection() throws SQLException + { + LOGGER.info("Attempting to connect to Trino via JDBC: {}", trinoJdbcUrl); + this.connection = DriverManager.getConnection(trinoJdbcUrl, trinoUser, trinoPassword); + LOGGER.info("Trino connection established successfully."); + } + + /** + * Ensures the current connection is valid, re-establishing it if necessary (closed, null, or invalid). + * @throws SQLException if connection cannot be re-established. + */ + private void ensureConnectionValid() throws SQLException { + if (connection == null || connection.isClosed() || !connection.isValid(5)) { + closeConnection(); + establishConnection(); + } + } + + /** + * Safely closes the current JDBC connection. + */ + private void closeConnection() { + if (connection != null) { + try { + connection.close(); + LOGGER.info("Trino connection closed."); + } catch (SQLException e) { + LOGGER.warn("Error closing Trino connection.", e); + } + connection = null; + } + } + + // ------------------------------------------------------------------------------------------------- + // Dynamic Table List Management + // ------------------------------------------------------------------------------------------------- + + /** + * Adds a table name to the monitoring list. + * This method can be called by external components (e.g., config trigger). + * @param tableName The name of the table to add. + */ + public void addMonitoredTable(String tableName) { + if (tableName == null || tableName.trim().isEmpty()) { + LOGGER.warn("Attempted to add null or empty table name to freshness monitor."); + return; + } + if (monitoredTables.add(tableName)) { + LOGGER.info("Table '{}' added to freshness monitor list.", tableName); + } else { + LOGGER.debug("Table '{}' is already in the freshness monitor list.", tableName); + } + } + + /** + * Removes a table name from the monitoring list. + * @param tableName The name of the table to remove. + */ + public void removeMonitoredTable(String tableName) { + if (monitoredTables.remove(tableName)) { + LOGGER.info("Table '{}' removed from freshness monitor list.", tableName); + } else { + LOGGER.debug("Table '{}' was not found in the freshness monitor list.", tableName); + } + } + + // ------------------------------------------------------------------------------------------------- + // Scheduling and Execution + // ------------------------------------------------------------------------------------------------- + + /** + * Starts the scheduled freshness monitoring task. + */ + public void start() { + try { + ensureConnectionValid(); + LOGGER.info("Starting Freshness Client, querying every {} seconds.", QUERY_INTERVAL_SECONDS); + + scheduler.scheduleAtFixedRate(this::queryAndCalculateFreshness, + 0, // Initial delay + QUERY_INTERVAL_SECONDS, + TimeUnit.SECONDS); + } catch (SQLException e) { + LOGGER.error("Failed to establish initial Trino connection. Freshness Client will not start.", e); + } + } + + + /** + * Stops the scheduled task and closes the JDBC connection. + */ + public void stop() { + LOGGER.info("Stopping Freshness Client."); + scheduler.shutdownNow(); + closeConnection(); + } + + /** + * The core scheduled task: queries max(freshness_ts) for all monitored tables + * and calculates the freshness metric. + */ + private void queryAndCalculateFreshness() { + try { + ensureConnectionValid(); + } catch (SQLException e) { + LOGGER.error("Connection failed during scheduled run. Skipping this freshness cycle.", e); + return; + } + + // Take a snapshot of the tables to monitor for this cycle. + // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. + Set tablesSnapshot = new HashSet<>(monitoredTables); + + for (String tableName : tablesSnapshot) { + + // Timestamp when the query is sent (t_send) + long tSendMillis = System.currentTimeMillis(); + + // Query to find the latest timestamp in the table + // Assumes 'freshness_ts' is a long-type epoch timestamp (milliseconds) + String query = String.format("SELECT max(freshness_ts) FROM %s", tableName); + + try (Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(query)) { + + // Timestamp when the result is received (t_receive) + long tReceiveMillis = System.currentTimeMillis(); + long maxFreshnessTs = 0; + + if (rs.next()) { + // Read the maximum timestamp value + maxFreshnessTs = rs.getLong(1); + } + + if (maxFreshnessTs > 0) { + // Freshness = t_receive - data_write_time (maxFreshnessTs) + // Result is in milliseconds + long freshnessMillis = tReceiveMillis - maxFreshnessTs; + + LOGGER.debug("Table {}: Max Ts: {}, Freshness: {} ms (Query RTT: {} ms)", + tableName, + maxFreshnessTs, + freshnessMillis, + tReceiveMillis - tSendMillis); + + // Record the calculated freshness using the MetricsFacade + metricsFacade.recordFreshness(freshnessMillis); + + } else { + LOGGER.warn("Table {} returned null or zero max(freshness_ts). Skipping freshness calculation.", tableName); + } + + } catch (SQLException e) { + // Handle database errors (e.g., table not found, query syntax error) + LOGGER.error("Failed to execute query for table {}: {}", tableName, e.getMessage()); + } catch (Exception e) { + // Catch potential runtime errors (e.g., in MetricsFacade) + LOGGER.error("Error calculating or recording freshness for table {}.", tableName, e); + } + } + } +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java new file mode 100644 index 0000000..da2c854 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -0,0 +1,61 @@ + /* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + + package io.pixelsdb.pixels.sink.util; + + import com.google.protobuf.ByteString; + import io.pixelsdb.pixels.retina.RetinaProto; + + import java.nio.ByteBuffer; + import java.util.List; + + public class DataTransform + { + public static ByteString longToByteString(long value) { + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); + return ByteString.copyFrom(bytes); + } + + public static void updateTimeStamp(List updateData, long txStartTime) { + ByteString timestampBytes = longToByteString(txStartTime); + + for (RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder : updateData) { + int insertDataCount = tableUpdateDataBuilder.getInsertDataCount(); + for (int i = 0; i < insertDataCount; i++) { + RetinaProto.InsertData.Builder insertBuilder = tableUpdateDataBuilder.getInsertDataBuilder(i); + int colValueCount = insertBuilder.getColValuesCount(); + if (colValueCount > 0) { + insertBuilder.setColValues(colValueCount - 1, timestampBytes); + } + } + + int updateDataCount = tableUpdateDataBuilder.getUpdateDataCount(); + for (int i = 0; i < updateDataCount; i++) { + RetinaProto.UpdateData.Builder updateBuilder = tableUpdateDataBuilder.getUpdateDataBuilder(i); + + int colValueCount = updateBuilder.getColValuesCount(); + if (colValueCount > 0) { + updateBuilder.setColValues(colValueCount - 1, timestampBytes); + } + } + tableUpdateDataBuilder.setTimestamp(txStartTime); + } + } + } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 0066db5..10ede82 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -23,6 +23,7 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.writer.retina.SinkContextManager; import io.prometheus.client.Counter; +import io.prometheus.client.Histogram; import io.prometheus.client.Summary; import lombok.Setter; import org.apache.commons.math3.stat.descriptive.SynchronizedDescriptiveStatistics; @@ -55,6 +56,8 @@ public class MetricsFacade private final Summary retinaServiceLatency; private final Summary writerLatency; private final Summary totalLatency; + private final Histogram transactionRowCountHistogram; + private final boolean monitorReportEnabled; private final int monitorReportInterval; private final int freshnessReportInterval; @@ -180,6 +183,11 @@ private MetricsFacade(boolean enabled) .quantile(0.95, 0.005) .quantile(0.99, 0.001) .register(); + this.transactionRowCountHistogram = Histogram.build() + .name("transaction_row_count_histogram") + .help("Distribution of row counts within a single transaction") + .buckets(1, 5, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100, 200) + .register(); this.freshness = new SynchronizedDescriptiveStatistics(); this.rowChangeSpeed = new SynchronizedDescriptiveStatistics(); @@ -201,6 +209,7 @@ private MetricsFacade(boolean enabled) this.totalLatency = null; this.freshness = null; this.rowChangeSpeed = null; + this.transactionRowCountHistogram = null; } freshnessReportInterval = config.getFreshnessReportInterval(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index fc9ae76..3fabb79 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -19,9 +19,11 @@ package io.pixelsdb.pixels.sink.writer.retina; +import com.google.protobuf.Message; import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.util.DataTransform; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,7 +82,7 @@ public void flush() List smallBatch = null; List txIds = new ArrayList<>(); List fullTableName = new ArrayList<>(); - List tableUpdateData = new LinkedList<>(); + List tableUpdateDataBuilderList = new LinkedList<>(); List tableUpdateCount = new ArrayList<>(); for (RowChangeEvent event : batch) { @@ -89,7 +91,7 @@ public void flush() { if (smallBatch != null && !smallBatch.isEmpty()) { - tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); + tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); tableUpdateCount.add(smallBatch.size()); } txIds.add(currTxId); @@ -103,16 +105,29 @@ public void flush() if (smallBatch != null) { // tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucketId(bucketId).setTxId(currentTxId).build()); - tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).build()); + tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); tableUpdateCount.add(smallBatch.size()); } flushRateLimiter.acquire(batch.size()); long txStartTime = System.currentTimeMillis(); + + if(freshnessLevel.equals("embed")) + { + DataTransform.updateTimeStamp(tableUpdateDataBuilderList, txStartTime); + } + + for(String writeTxId: txIds) { sinkContextManager.getSinkContext(writeTxId).setCurrStartTime(); } + + List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); + for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) + { + tableUpdateData.add(tableUpdateDataItem.build()); + } CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); updateRecordResponseCompletableFuture.thenAccept( From 7b8359af4c5db8120453bc9f4af397311ba54758 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 28 Nov 2025 12:44:13 +0800 Subject: [PATCH 31/53] Stats --- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 2 +- .../sink/freshness/FreshnessClient.java | 16 +++++- ...leProviderAndProcessorPipelineManager.java | 17 ++++++ .../pixels/sink/util/MetricsFacade.java | 55 ++++++++++++++++++- .../pixels/sink/writer/NoneWriter.java | 15 +++++ 5 files changed, 102 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 5e2d044..9689bda 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -93,7 +93,7 @@ public static void main(String[] args) throws IOException if (config.getSinkMonitorFreshnessLevel().equals("embed")) { - freshnessClient = new FreshnessClient(); + freshnessClient = FreshnessClient.getInstance(); freshnessClient.start(); } sinkSource.start(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index c601585..e7ca7dc 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -51,8 +51,9 @@ public class FreshnessClient { private Connection connection; private final ScheduledExecutorService scheduler; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private static volatile FreshnessClient instance; - public FreshnessClient() { + private FreshnessClient() { // Initializes the set with thread safety wrapper this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); @@ -65,6 +66,19 @@ public FreshnessClient() { }); } + public static FreshnessClient getInstance() { + if (instance == null) { + // First check: Reduces synchronization overhead once the instance is created + synchronized (FreshnessClient.class) { + if (instance == null) { + // Second check: Only one thread proceeds to create the instance + instance = new FreshnessClient(); + } + } + } + return instance; + } + // ------------------------------------------------------------------------------------------------- // Connection Management // ------------------------------------------------------------------------------------------------- diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index 304c82d..664b4f9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -20,6 +20,12 @@ import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; +import io.pixelsdb.pixels.sink.metadata.TableMetadata; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TableProcessor; import org.apache.kafka.connect.source.SourceRecord; @@ -57,6 +63,17 @@ public void routeRecord(Integer tableId, SOURCE_RECORD_T record) ); tableProcessor.run(); newPipeline.run(); + if(PixelsSinkConfigFactory.getInstance().getSinkMonitorFreshnessLevel().equals("embed")) + { + try + { + String tableName = TableMetadataRegistry.Instance().getSchemaTableName(tableId).getTableName(); + FreshnessClient.getInstance().addMonitoredTable(tableName); + } catch (SinkException e) + { + throw new RuntimeException(e); + } + } return newPipeline; }); pipeline.putRawEvent(record); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 10ede82..4c15d2c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -17,6 +17,7 @@ package io.pixelsdb.pixels.sink.util; +import com.google.protobuf.ByteString; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; @@ -32,6 +33,7 @@ import java.io.FileWriter; import java.io.IOException; +import java.nio.ByteBuffer; import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; import java.util.concurrent.atomic.AtomicBoolean; @@ -57,6 +59,7 @@ public class MetricsFacade private final Summary writerLatency; private final Summary totalLatency; private final Histogram transactionRowCountHistogram; + private final Histogram primaryKeyUpdateDistribution; private final boolean monitorReportEnabled; private final int monitorReportInterval; @@ -188,7 +191,12 @@ private MetricsFacade(boolean enabled) .help("Distribution of row counts within a single transaction") .buckets(1, 5, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100, 200) .register(); - + this.primaryKeyUpdateDistribution = Histogram.build() + .name("primary_key_update_distribution") + .help("Distribution of primary key updates by logical bucket/hash for hot spot analysis") + .labelNames("table") // Table name tag + .buckets(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) // 10 buckets for distribution + .register(); this.freshness = new SynchronizedDescriptiveStatistics(); this.rowChangeSpeed = new SynchronizedDescriptiveStatistics(); } else @@ -210,6 +218,7 @@ private MetricsFacade(boolean enabled) this.freshness = null; this.rowChangeSpeed = null; this.transactionRowCountHistogram = null; + this.primaryKeyUpdateDistribution = null; } freshnessReportInterval = config.getFreshnessReportInterval(); @@ -405,6 +414,50 @@ public void recordFreshness(double freshnessMill) } } + public void recordPrimaryKeyUpdateDistribution(String table, ByteString pkValue) { + if (!enabled || primaryKeyUpdateDistribution == null) { + return; + } + if (pkValue == null || pkValue.isEmpty()) { + LOGGER.debug("Skipping PK distribution recording: pkValue is null or empty for table {}.", table); + return; + } + + long numericPK; + int length = pkValue.size(); + + try { + ByteBuffer buffer = pkValue.asReadOnlyByteBuffer(); + + if (length == Integer.BYTES) { + numericPK = Integer.toUnsignedLong(buffer.getInt()); + } else if (length == Long.BYTES) { + numericPK = buffer.getLong(); + } else { + LOGGER.warn("Unsupported PK ByteString length {} for table {}. Expected 4 or 8.", length, table); + return; + } + } catch (Exception e) { + LOGGER.error("Failed to convert ByteString to numeric type for table {}: {}", table, e.getMessage()); + return; + } + int hash = Long.hashCode(numericPK); + double bucketIndex = (Math.abs(hash % 10)) + 1; + + // 3. 记录到 Histogram + primaryKeyUpdateDistribution.labels(table).observe(bucketIndex); + + LOGGER.debug("Table {}: PK {} mapped to bucket index {}", table, numericPK, bucketIndex); + } + public void recordTransactionRowCount(int rowCount) + { + if (enabled && transactionRowCountHistogram != null) + { + // Use observe() to add the value to the Histogram's configured buckets. + transactionRowCountHistogram.observe(rowCount); + } + } + public void run() { while (running.get()) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index 1514c2d..31a9208 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -19,6 +19,7 @@ import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.util.MetricsFacade; import java.io.IOException; @@ -38,6 +39,16 @@ public void flush() public boolean writeRow(RowChangeEvent rowChangeEvent) { metricsFacade.recordRowEvent(); + metricsFacade.recordRowChange(rowChangeEvent.getTable(), rowChangeEvent.getOp()); + try + { + rowChangeEvent.initIndexKey(); + metricsFacade.recordPrimaryKeyUpdateDistribution(rowChangeEvent.getTable(), rowChangeEvent.getAfterKey().getKey()); + } catch (SinkException e) + { + throw new RuntimeException(e); + } + return true; } @@ -47,6 +58,10 @@ public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) { metricsFacade.recordTransaction(); + for(SinkProto.DataCollection dataCollection: transactionMetadata.getDataCollectionsList()) + { + metricsFacade.recordTransactionRowCount((int) dataCollection.getEventCount()); + } } return true; } From 383d4e4516dacfc12fae49708bfbfa14723c7672 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 28 Nov 2025 10:00:36 +0000 Subject: [PATCH 32/53] fix freshness client --- conf/pixels-sink.aws.properties | 23 +++--- pom.xml | 7 +- .../pixels/sink/config/PixelsSinkConfig.java | 9 +++ .../sink/freshness/FreshnessClient.java | 70 ++++++++++++------- .../pixels/sink/processor/TableProcessor.java | 12 +++- ...leProviderAndProcessorPipelineManager.java | 11 --- .../pixels/sink/util/DataTransform.java | 1 - .../pixels/sink/writer/NoneWriter.java | 4 +- .../sink/writer/retina/SinkContext.java | 3 +- .../writer/retina/TableCrossTxWriter.java | 5 +- .../sink/freshness/TestFreshnessClient.java | 52 ++++++++++++++ 11 files changed, 146 insertions(+), 51 deletions(-) create mode 100644 src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index a2c079c..fefa049 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,14 +1,19 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=150000 +sink.datasource.rate.limit=10000 # Sink Config: retina | csv | proto | none sink.mode=retina sink.retina.client=2 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate150k_6.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness150k_6.csv -# row or txn -sink.monitor.freshness.level=row +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_test.csv +# trino for freshness query +trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x +trino.user=pixels +trino.password=password + +# row or txn or embed +sink.monitor.freshness.level=embed sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -41,8 +46,8 @@ sink.commit.batch.size=200 sink.commit.batch.worker=32 sink.commit.batch.delay=1000 ## Proto Config -sink.proto.dir=file:///home/ubuntu/pixels-sink/tmp -sink.proto.data=hybench10_2 +sink.proto.dir=file:///home/ubuntu/disk1/hybench +sink.proto.data=hybench10_5 sink.proto.maxRecords=1000000 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 @@ -50,13 +55,13 @@ sink.registry.url=http://localhost:8080/apis/registry/v2 transaction.topic.suffix=transaction #transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer -sink.trans.batch.size=200 +sink.trans.batch.size=100 ## Batch or trans or record sink.trans.mode=batch # Sink Metrics sink.monitor.enable=true sink.monitor.port=9465 -sink.monitor.report.interval=1000 +sink.monitor.report.interval=10000 sink.monitor.freshness.interval=1000 # Interact with other rpc diff --git a/pom.xml b/pom.xml index 0483d9f..a4919eb 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,7 @@ 1.18.42 3.2.3.Final 1.4.13 + 440 @@ -72,7 +73,11 @@ io.grpc grpc-netty - + + io.trino + trino-jdbc + ${trino.version} + com.alibaba fastjson diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index d5f60cb..9cee533 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -173,6 +173,15 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.freshness.level", defaultValue = "row") // row or txn or embed private String sinkMonitorFreshnessLevel; + @ConfigKey(value = "trino.url") + private String trinoUrl; + + @ConfigKey(value = "trino.user") + private String trinoUser; + + @ConfigKey(value = "trino.password") + private String trinoPassword; + public PixelsSinkConfig(String configFilePath) throws IOException { this.config = ConfigFactory.Instance(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index e7ca7dc..5806a71 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -20,14 +20,14 @@ package io.pixelsdb.pixels.sink.freshness; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.*; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; +import java.util.*; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -40,9 +40,9 @@ public class FreshnessClient { private static final Logger LOGGER = LoggerFactory.getLogger(FreshnessClient.class); // Configuration parameters (should ideally be loaded from a config file) - private final String trinoJdbcUrl = "jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x"; - private final String trinoUser = "pixels"; - private final String trinoPassword = "password"; + private final String trinoJdbcUrl; + private final String trinoUser; + private final String trinoPassword; // Key modification: Use a thread-safe Set to maintain the list of tables to monitor dynamically. private final Set monitoredTables; @@ -57,6 +57,11 @@ private FreshnessClient() { // Initializes the set with thread safety wrapper this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + this.trinoUser = config.getTrinoUser(); + this.trinoJdbcUrl = config.getTrinoUrl(); + this.trinoPassword = config.getTrinoPassword(); + // Initializes a single-threaded scheduler for executing freshness queries this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { Thread t = Executors.defaultThreadFactory().newThread(r); @@ -90,7 +95,14 @@ public static FreshnessClient getInstance() { private void establishConnection() throws SQLException { LOGGER.info("Attempting to connect to Trino via JDBC: {}", trinoJdbcUrl); - this.connection = DriverManager.getConnection(trinoJdbcUrl, trinoUser, trinoPassword); + try + { + Class.forName("io.trino.jdbc.TrinoDriver"); + } catch (ClassNotFoundException e) + { + throw new SQLException(e); + } + this.connection = DriverManager.getConnection(trinoJdbcUrl, trinoUser, null); LOGGER.info("Trino connection established successfully."); } @@ -188,7 +200,7 @@ public void stop() { * The core scheduled task: queries max(freshness_ts) for all monitored tables * and calculates the freshness metric. */ - private void queryAndCalculateFreshness() { + void queryAndCalculateFreshness() { try { ensureConnectionValid(); } catch (SQLException e) { @@ -199,8 +211,26 @@ private void queryAndCalculateFreshness() { // Take a snapshot of the tables to monitor for this cycle. // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. Set tablesSnapshot = new HashSet<>(monitoredTables); + if (tablesSnapshot.isEmpty()) { + LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); + return; + } + + String tableName; + try { + List tableList = new ArrayList<>(tablesSnapshot); + + Random random = new Random(); + int randomIndex = random.nextInt(tableList.size()); - for (String tableName : tablesSnapshot) { + tableName = tableList.get(randomIndex); + + LOGGER.debug("Randomly selected table for this cycle: {}", tableName); + + } catch (Exception e) { + LOGGER.error("Error selecting a random table from the monitor list.", e); + return; + } // Timestamp when the query is sent (t_send) long tSendMillis = System.currentTimeMillis(); @@ -212,27 +242,17 @@ private void queryAndCalculateFreshness() { try (Statement statement = connection.createStatement(); ResultSet rs = statement.executeQuery(query)) { - // Timestamp when the result is received (t_receive) - long tReceiveMillis = System.currentTimeMillis(); - long maxFreshnessTs = 0; + Timestamp maxFreshnessTs = null; if (rs.next()) { // Read the maximum timestamp value - maxFreshnessTs = rs.getLong(1); + maxFreshnessTs = rs.getTimestamp(1); } - if (maxFreshnessTs > 0) { - // Freshness = t_receive - data_write_time (maxFreshnessTs) + if (maxFreshnessTs != null) { + // Freshness = t_send - data_write_time (maxFreshnessTs) // Result is in milliseconds - long freshnessMillis = tReceiveMillis - maxFreshnessTs; - - LOGGER.debug("Table {}: Max Ts: {}, Freshness: {} ms (Query RTT: {} ms)", - tableName, - maxFreshnessTs, - freshnessMillis, - tReceiveMillis - tSendMillis); - - // Record the calculated freshness using the MetricsFacade + long freshnessMillis = tSendMillis - maxFreshnessTs.getTime(); metricsFacade.recordFreshness(freshnessMillis); } else { @@ -246,6 +266,6 @@ private void queryAndCalculateFreshness() { // Catch potential runtime errors (e.g., in MetricsFacade) LOGGER.error("Error calculating or recording freshness for table {}.", tableName, e); } - } + monitoredTables.clear(); } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 3df44f1..9a91030 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -19,7 +19,11 @@ package io.pixelsdb.pixels.sink.processor; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; +import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.provider.TableEventProvider; import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; @@ -43,7 +47,7 @@ public class TableProcessor implements StoppableProcessor, Runnable private final TableEventProvider tableEventProvider; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private Thread processorThread; - + private boolean tableAdded = false; public TableProcessor(TableEventProvider tableEventProvider) { this.pixelsSinkWriter = PixelsSinkWriterFactory.getWriter(); @@ -66,6 +70,12 @@ private void processLoop() { continue; } + if(!tableAdded && + PixelsSinkConfigFactory.getInstance().getSinkMonitorFreshnessLevel().equals("embed")) + { + tableAdded = true; + FreshnessClient.getInstance().addMonitoredTable(event.getTable()); + } pixelsSinkWriter.writeRow(event); } LOGGER.info("Processor thread exited"); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index 664b4f9..af844b0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -63,17 +63,6 @@ public void routeRecord(Integer tableId, SOURCE_RECORD_T record) ); tableProcessor.run(); newPipeline.run(); - if(PixelsSinkConfigFactory.getInstance().getSinkMonitorFreshnessLevel().equals("embed")) - { - try - { - String tableName = TableMetadataRegistry.Instance().getSchemaTableName(tableId).getTableName(); - FreshnessClient.getInstance().addMonitoredTable(tableName); - } catch (SinkException e) - { - throw new RuntimeException(e); - } - } return newPipeline; }); pipeline.putRawEvent(record); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java index da2c854..a0a14d9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -55,7 +55,6 @@ public static void updateTimeStamp(List upd updateBuilder.setColValues(colValueCount - 1, timestampBytes); } } - tableUpdateDataBuilder.setTimestamp(txStartTime); } } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index 31a9208..a43242f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -58,10 +58,12 @@ public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) { metricsFacade.recordTransaction(); + int cnt = 0; for(SinkProto.DataCollection dataCollection: transactionMetadata.getDataCollectionsList()) { - metricsFacade.recordTransactionRowCount((int) dataCollection.getEventCount()); + cnt += dataCollection.getEventCount(); } + metricsFacade.recordTransactionRowCount(cnt); } return true; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 42c728b..8c74d15 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -107,7 +107,8 @@ boolean isCompleted(SinkProto.TransactionMetadata tx) Long targetEventCount = tableCounters.get(dataCollection.getDataCollection()); long target = targetEventCount == null ? 0 : targetEventCount; LOGGER.debug("TX {}, Table {}, event count {}, tableCursors {}", tx.getId(), dataCollection.getDataCollection(), dataCollection.getEventCount(), target); - if (dataCollection.getEventCount() > target) +// if (dataCollection.getEventCount() > target) + if(false) { return false; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 3fabb79..a2106c2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -23,6 +23,7 @@ import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; import io.pixelsdb.pixels.sink.util.DataTransform; import lombok.Getter; import org.slf4j.Logger; @@ -114,7 +115,9 @@ public void flush() if(freshnessLevel.equals("embed")) { - DataTransform.updateTimeStamp(tableUpdateDataBuilderList, txStartTime); + long freshness_ts = txStartTime * 1000; + FreshnessClient.getInstance().addMonitoredTable(tableName); + DataTransform.updateTimeStamp(tableUpdateDataBuilderList, freshness_ts); } diff --git a/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java b/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java new file mode 100644 index 0000000..4123b29 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java @@ -0,0 +1,52 @@ +package io.pixelsdb.pixels.sink.freshness; + +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.util.MetricsFacade; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.concurrent.ScheduledExecutorService; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.*; + +// We extend FreshnessClient to access the protected queryAndCalculateFreshness method +public class TestFreshnessClient +{ + + // Mocks for JDBC dependencies + private Connection mockConnection; + private Statement mockStatement; + private ResultSet mockResultSet; + + // Mocks for utility/config dependencies + private PixelsSinkConfig mockConfig; + private MetricsFacade mockMetricsFacade; + private FreshnessClient client; // The instance of the client to test + + @BeforeAll + public static void setUp() throws IOException + { + // Initialization as per the user's template + PixelsSinkConfigFactory.initialize("/home/ubuntu/pixels-sink/conf/pixels-sink.aws.properties"); + } + @Test + public void testFreshnessCalculationSuccess() throws Exception { + + FreshnessClient freshnessClient = FreshnessClient.getInstance(); + freshnessClient.addMonitoredTable("customer"); + freshnessClient.start(); + while(true){} + } +} \ No newline at end of file From 6d86f978102d10309a8798db51f22e1c647ba2c1 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 28 Nov 2025 10:03:25 +0000 Subject: [PATCH 33/53] fix sinkcontext --- .../io/pixelsdb/pixels/sink/writer/retina/SinkContext.java | 3 +-- .../pixels/sink/writer/retina/TableCrossTxWriter.java | 5 ++++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 8c74d15..42c728b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -107,8 +107,7 @@ boolean isCompleted(SinkProto.TransactionMetadata tx) Long targetEventCount = tableCounters.get(dataCollection.getDataCollection()); long target = targetEventCount == null ? 0 : targetEventCount; LOGGER.debug("TX {}, Table {}, event count {}, tableCursors {}", tx.getId(), dataCollection.getDataCollection(), dataCollection.getEventCount(), target); -// if (dataCollection.getEventCount() > target) - if(false) + if (dataCollection.getEventCount() > target) { return false; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index a2106c2..ba9a934 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -173,7 +173,10 @@ private void updateCtxCounters(List txIds, List fullTableName, L metricsFacade.recordRowEvent(tableUpdateCount.get(i)); String writeTxId = txIds.get(i); SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); - sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); + if(sinkContext !=null) + { + sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); + } } writeLock.unlock(); } From 26880b526a74a9eee2d27603b26ab58c2a4e527b Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 28 Nov 2025 10:21:26 +0000 Subject: [PATCH 34/53] update ratelimiter --- conf/pixels-sink.aws.properties | 6 +- .../sink/freshness/FreshnessClient.java | 5 -- .../pixels/sink/util/FlushRateLimiter.java | 69 +++++++++++++++++-- .../writer/retina/TableCrossTxWriter.java | 3 +- 4 files changed, 69 insertions(+), 14 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index fefa049..6d2e363 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -13,7 +13,7 @@ trino.user=pixels trino.password=password # row or txn or embed -sink.monitor.freshness.level=embed +sink.monitor.freshness.level=txn sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -37,11 +37,11 @@ sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 sink.flush.interval.ms=20000 -sink.flush.batch.size=500 +sink.flush.batch.size=100 sink.max.retries=3 ## writer commit # sync or async -sink.commit.method=async +sink.commit.method=sync sink.commit.batch.size=200 sink.commit.batch.worker=32 sink.commit.batch.delay=1000 diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 5806a71..1b8d49d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -146,11 +146,6 @@ public void addMonitoredTable(String tableName) { LOGGER.warn("Attempted to add null or empty table name to freshness monitor."); return; } - if (monitoredTables.add(tableName)) { - LOGGER.info("Table '{}' added to freshness monitor list.", tableName); - } else { - LOGGER.debug("Table '{}' is already in the freshness monitor list.", tableName); - } } /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java index f59640f..061e3d3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java @@ -20,10 +20,24 @@ import com.google.common.util.concurrent.RateLimiter; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; public class FlushRateLimiter { - private final RateLimiter rateLimiter; + private static final Logger LOGGER = LoggerFactory.getLogger(FlushRateLimiter.class); + + private final Semaphore semaphore; private final boolean enableRateLimiter; + private final ScheduledExecutorService scheduler; + + // Configuration derived parameters + private static final long REFRESH_PERIOD_MS = 10; + private final int replenishmentAmount; private static volatile FlushRateLimiter instance; @@ -31,8 +45,42 @@ private FlushRateLimiter() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); int sourceRateLimit = pixelsSinkConfig.getSourceRateLimit(); - this.rateLimiter = RateLimiter.create(sourceRateLimit); this.enableRateLimiter = pixelsSinkConfig.isEnableSourceRateLimit(); + + if (sourceRateLimit <= 0 || !enableRateLimiter) { + this.semaphore = null; + this.replenishmentAmount = 0; + this.scheduler = null; + return; + } + + double replenishmentPerMillisecond = (double) sourceRateLimit / 1000.0; + this.replenishmentAmount = (int) Math.max(1, Math.round(replenishmentPerMillisecond * REFRESH_PERIOD_MS)); + + this.semaphore = new Semaphore(this.replenishmentAmount); + + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = Executors.defaultThreadFactory().newThread(r); + t.setName("Rate-Limiter-Replenish"); + t.setDaemon(true); + return t; + }); + + this.scheduler.scheduleAtFixedRate( + this::replenishTokens, + REFRESH_PERIOD_MS, + REFRESH_PERIOD_MS, + TimeUnit.MILLISECONDS + ); + + LOGGER.info("FlushRateLimiter initialized. Rate: {}/s, Replenishment: {} tokens every {}ms.", + sourceRateLimit, this.replenishmentAmount, REFRESH_PERIOD_MS); + } + + private void replenishTokens() { + if (semaphore != null) { + semaphore.release(replenishmentAmount); + } } public static FlushRateLimiter getInstance() @@ -52,9 +100,20 @@ public static FlushRateLimiter getInstance() public void acquire(int num) { - if (enableRateLimiter) { - rateLimiter.acquire(num); + if (enableRateLimiter && semaphore != null) { + try { + semaphore.acquire(num); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("FlushRateLimiter acquire interrupted.", e); + } } } -} + public void shutdown() { + if (scheduler != null) { + scheduler.shutdownNow(); + LOGGER.info("FlushRateLimiter scheduler stopped."); + } + } +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index ba9a934..28797d2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -113,7 +113,8 @@ public void flush() flushRateLimiter.acquire(batch.size()); long txStartTime = System.currentTimeMillis(); - if(freshnessLevel.equals("embed")) +// if(freshnessLevel.equals("embed")) + if(true) { long freshness_ts = txStartTime * 1000; FreshnessClient.getInstance().addMonitoredTable(tableName); From cf868557dd8ec15fc8b0da367325320ac61860c7 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 28 Nov 2025 10:25:56 +0000 Subject: [PATCH 35/53] RateLimit Transaction --- conf/pixels-sink.aws.properties | 2 +- .../io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java | 5 ++++- .../pixels/sink/writer/retina/TableCrossTxWriter.java | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 6d2e363..9cc3e47 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,7 +1,7 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=10000 +sink.datasource.rate.limit=500 # Sink Config: retina | csv | proto | none sink.mode=retina sink.retina.client=2 diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index d6c89d9..2dcf7b4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -24,6 +24,7 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import io.prometheus.client.Summary; @@ -46,13 +47,14 @@ public class RetinaWriter implements PixelsSinkWriter private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(2048); private final ScheduledExecutorService timeoutScheduler = Executors.newSingleThreadScheduledExecutor(); - + private final FlushRateLimiter flushRateLimiter; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final SinkContextManager sinkContextManager; public RetinaWriter() { this.sinkContextManager = SinkContextManager.getInstance(); + this.flushRateLimiter = FlushRateLimiter.getInstance(); } @Override @@ -120,6 +122,7 @@ private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkExc // startTrans(txBegin.getId()).get(); try { + flushRateLimiter.acquire(1); startTransSync(txBegin.getId()); } catch (SinkException e) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 28797d2..0cf0ee7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -110,7 +110,7 @@ public void flush() tableUpdateCount.add(smallBatch.size()); } - flushRateLimiter.acquire(batch.size()); + // flushRateLimiter.acquire(batch.size()); long txStartTime = System.currentTimeMillis(); // if(freshnessLevel.equals("embed")) From e4b26693621dcd09e87ad0d763d77bd4431a0c39 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Fri, 28 Nov 2025 21:57:07 +0000 Subject: [PATCH 36/53] Refine NoneWriter And ProtoWriter --- conf/pixels-sink.aws.properties | 8 +- .../sink/freshness/FreshnessClient.java | 1 + .../sink/processor/TransactionProcessor.java | 7 +- .../TableEventStorageLoopProvider.java | 43 ++++ ...leProviderAndProcessorPipelineManager.java | 5 + .../TransactionEventStorageLoopProvider.java | 44 ++++ .../source/AbstractSinkStorageSource.java | 130 ++++++---- .../sink/source/LegacySinkStorageSource.java | 4 +- .../pixels/sink/util/FlushRateLimiter.java | 5 + .../pixels/sink/util/TableCounters.java | 37 +++ .../pixels/sink/writer/NoneWriter.java | 231 ++++++++++++++++- .../pixels/sink/writer/ProtoWriter.java | 242 +++++++++++++++--- .../sink/writer/retina/RetinaWriter.java | 2 +- .../sink/writer/retina/SinkContext.java | 1 + .../writer/retina/SinkContextManager.java | 19 +- .../writer/retina/TableCrossTxWriter.java | 35 ++- 16 files changed, 707 insertions(+), 107 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 9cc3e47..26298a8 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,9 +1,9 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=500 +sink.datasource.rate.limit=30000 # Sink Config: retina | csv | proto | none -sink.mode=retina +sink.mode=none sink.retina.client=2 sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_test.csv @@ -14,7 +14,7 @@ trino.password=password # row or txn or embed sink.monitor.freshness.level=txn -sink.storage.loop=true +sink.storage.loop=false # Kafka Config bootstrap.servers=realtime-kafka-2:29092 group.id=3078 @@ -47,7 +47,7 @@ sink.commit.batch.worker=32 sink.commit.batch.delay=1000 ## Proto Config sink.proto.dir=file:///home/ubuntu/disk1/hybench -sink.proto.data=hybench10_5 +sink.proto.data=hybench10_7 sink.proto.maxRecords=1000000 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 1b8d49d..8f142a6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -146,6 +146,7 @@ public void addMonitoredTable(String tableName) { LOGGER.warn("Attempted to add null or empty table name to freshness monitor."); return; } + monitoredTables.add(tableName); } /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java index d5b3597..3a50a46 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -46,7 +46,12 @@ public void run() while (running.get()) { SinkProto.TransactionMetadata transaction = transactionEventProvider.getTransaction(); - LOGGER.trace("Processing transaction event: {}", transaction.getId()); + if(transaction == null) + { + LOGGER.warn("Received null transaction"); + running.set(false); + break; + } sinkWriter.writeTrans(transaction); } LOGGER.info("Processor thread exited for transaction"); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java new file mode 100644 index 0000000..d30bb36 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java @@ -0,0 +1,43 @@ +package io.pixelsdb.pixels.sink.provider; + +import com.google.protobuf.InvalidProtocolBufferException; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; +import io.pixelsdb.pixels.sink.exception.SinkException; + +import java.nio.ByteBuffer; +import java.util.logging.Logger; +import io.pixelsdb.pixels.core.utils.Pair; +public class TableEventStorageLoopProvider extends TableEventProvider +{ + private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); + + protected TableEventStorageLoopProvider() + { + super(); + } + + @Override + RowChangeEvent convertToTargetRecord(T record) + { + Pair pairRecord = (Pair) record; + ByteBuffer sourceRecord = pairRecord.getLeft(); + Integer loopId = pairRecord.getRight(); + try + { + SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); + + SinkProto.RowRecord.Builder rowRecordBuilder = rowRecord.toBuilder(); + SinkProto.TransactionInfo.Builder transactionBuilder = rowRecordBuilder.getTransactionBuilder(); + String id = transactionBuilder.getId(); + transactionBuilder.setId(id + "_" + Integer.toString(loopId)); + rowRecordBuilder.setTransaction(transactionBuilder); + return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecordBuilder.build()); + } catch (InvalidProtocolBufferException | SinkException e) + { + LOGGER.warning(e.getMessage()); + return null; + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index af844b0..f8afd6f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -20,6 +20,7 @@ import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.exception.SinkException; @@ -71,6 +72,10 @@ public void routeRecord(Integer tableId, SOURCE_RECORD_T record) private TableEventProvider createProvider(SOURCE_RECORD_T record) { Class recordType = record.getClass(); + if (recordType == Pair.class) + { + return new TableEventStorageLoopProvider<>(); + } if (recordType == SourceRecord.class) { return new TableEventEngineProvider<>(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java new file mode 100644 index 0000000..afa0b38 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java @@ -0,0 +1,44 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package io.pixelsdb.pixels.sink.provider; + +import com.google.protobuf.InvalidProtocolBufferException; +import io.pixelsdb.pixels.core.utils.Pair; +import io.pixelsdb.pixels.sink.SinkProto; + +import java.nio.ByteBuffer; + +public class TransactionEventStorageLoopProvider extends TransactionEventProvider +{ + @Override + SinkProto.TransactionMetadata convertToTargetRecord(T record) + { + Pair buffer = (Pair) record; + try + { + SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(buffer.getLeft()); + Integer loopId = buffer.getRight(); + SinkProto.TransactionMetadata.Builder builder = tx.toBuilder(); + builder.setId(builder.getId() + "_" + loopId); + return builder.build(); + } catch (InvalidProtocolBufferException e) + { + throw new RuntimeException(e); + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index 470eb99..fb04de9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -1,17 +1,19 @@ package io.pixelsdb.pixels.sink.source; -import com.google.common.util.concurrent.RateLimiter; import io.pixelsdb.pixels.common.physical.PhysicalReader; import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; import io.pixelsdb.pixels.sink.provider.ProtoType; import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; +import io.pixelsdb.pixels.sink.provider.TransactionEventStorageLoopProvider; import io.pixelsdb.pixels.sink.provider.TransactionEventStorageProvider; import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; +import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -25,7 +27,8 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; -public abstract class AbstractSinkStorageSource implements SinkSource { +public abstract class AbstractSinkStorageSource implements SinkSource +{ private static final Logger LOGGER = LoggerFactory.getLogger(AbstractSinkStorageSource.class); protected final AtomicBoolean running = new AtomicBoolean(false); @@ -36,17 +39,20 @@ public abstract class AbstractSinkStorageSource implements SinkSource { protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); private final Map consumerThreads = new ConcurrentHashMap<>(); private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - private final Map>> queueMap = new ConcurrentHashMap<>(); + private final Map, Integer>>> queueMap = new ConcurrentHashMap<>(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final TableProviderAndProcessorPipelineManager tablePipelineManager = new TableProviderAndProcessorPipelineManager(); + private final TableProviderAndProcessorPipelineManager> tablePipelineManager = new TableProviderAndProcessorPipelineManager<>(); private final boolean storageLoopEnabled; private final int MAX_QUEUE_SIZE = 10_000; - protected TransactionEventStorageProvider transactionEventProvider; + private final FlushRateLimiter sourceRateLimiter; + protected TransactionEventStorageLoopProvider> transactionEventProvider; protected TransactionProcessor transactionProcessor; protected Thread transactionProviderThread; protected Thread transactionProcessorThread; + private int loopId = 0; - protected AbstractSinkStorageSource() { + protected AbstractSinkStorageSource() + { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); this.topic = pixelsSinkConfig.getSinkProtoData(); this.baseDir = pixelsSinkConfig.getSinkProtoDir(); @@ -54,48 +60,61 @@ protected AbstractSinkStorageSource() { this.files = this.etcdFileRegistry.listAllFiles(); this.storageLoopEnabled = pixelsSinkConfig.isSinkStorageLoop(); - this.transactionEventProvider = new TransactionEventStorageProvider<>(); + this.transactionEventProvider = new TransactionEventStorageLoopProvider<>(); this.transactionProviderThread = new Thread(transactionEventProvider); this.transactionProcessor = new TransactionProcessor(transactionEventProvider); this.transactionProcessorThread = new Thread(transactionProcessor, "debezium-processor"); + this.sourceRateLimiter = FlushRateLimiter.getNewInstance(); } abstract ProtoType getProtoType(int i); - protected void handleTransactionSourceRecord(ByteBuffer record) { - transactionEventProvider.putTransRawEvent(record); + protected void handleTransactionSourceRecord(ByteBuffer record, Integer loopId) + { + sourceRateLimiter.acquire(1); + transactionEventProvider.putTransRawEvent(new Pair<>(record, loopId)); } @Override - public void start() { + public void start() + { this.running.set(true); this.transactionProcessorThread.start(); this.transactionProviderThread.start(); List readers = new ArrayList<>(); - for (String file : files) { + for (String file : files) + { Storage.Scheme scheme = Storage.Scheme.fromPath(file); LOGGER.info("Start read from file {}", file); PhysicalReader reader = null; - try { + try + { reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); - } catch (IOException e) { + } catch (IOException e) + { throw new RuntimeException(e); } readers.add(reader); } - do { - for (PhysicalReader reader : readers) { + do + { + for (PhysicalReader reader : readers) + { LOGGER.info("Start Read {}", reader.getPath()); long offset = 0; - while (true) { - try { + while (true) + { + try + { int key, valueLen; reader.seek(offset); - try { + try + { key = reader.readInt(ByteOrder.BIG_ENDIAN); valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) { + } catch (IOException e) + { // EOF break; } @@ -110,12 +129,12 @@ public void start() { // Get or create queue - BlockingQueue> queue = + BlockingQueue, Integer>> queue = queueMap.computeIfAbsent(key, k -> new LinkedBlockingQueue<>(MAX_QUEUE_SIZE)); // Put future in queue - queue.put(valueFuture); + queue.put(new Pair<>(valueFuture, loopId)); // Start consumer thread if not exists consumerThreads.computeIfAbsent(key, k -> @@ -125,19 +144,23 @@ public void start() { t.start(); return t; }); - } catch (IOException | InterruptedException e) { + } catch (IOException | InterruptedException e) + { break; } } } + ++loopId; } while (storageLoopEnabled && isRunning()); // signal all queues to stop queueMap.values().forEach(q -> { - try { - q.put(POISON_PILL); - } catch (InterruptedException e) { + try + { + q.put(new Pair<>(POISON_PILL, loopId)); + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } }); @@ -145,47 +168,61 @@ public void start() { // wait all consumers to finish consumerThreads.values().forEach(t -> { - try { + try + { t.join(); - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); } }); // close all readers - for(PhysicalReader reader: readers) + for (PhysicalReader reader : readers) { - try { + try + { reader.close(); - } catch (IOException e) { + } catch (IOException e) + { throw new RuntimeException(e); } } } - private void consumeQueue(int key, BlockingQueue> queue, ProtoType protoType) { - try { - while (true) { - CompletableFuture value = queue.take(); - if (value == POISON_PILL) { + private void consumeQueue(int key, BlockingQueue, Integer>> queue, ProtoType protoType) + { + try + { + while (true) + { + Pair, Integer> pair = queue.take(); + CompletableFuture value = pair.getLeft(); + int loopId = pair.getRight(); + if (value == POISON_PILL) + { break; } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - switch (protoType) { - case ROW -> handleRowChangeSourceRecord(key, valueBuffer); - case TRANS -> handleTransactionSourceRecord(valueBuffer); + switch (protoType) + { + case ROW -> handleRowChangeSourceRecord(key, valueBuffer, loopId); + case TRANS -> handleTransactionSourceRecord(valueBuffer, loopId); } } - } catch (InterruptedException e) { + } catch (InterruptedException e) + { Thread.currentThread().interrupt(); - } catch (ExecutionException e) { + } catch (ExecutionException e) + { LOGGER.error("Error in async processing", e); } } - private ByteBuffer copyToHeap(ByteBuffer directBuffer) { + private ByteBuffer copyToHeap(ByteBuffer directBuffer) + { ByteBuffer duplicate = directBuffer.duplicate(); ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); heapBuffer.put(duplicate); @@ -193,17 +230,20 @@ private ByteBuffer copyToHeap(ByteBuffer directBuffer) { return heapBuffer; } - private void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer) { - tablePipelineManager.routeRecord(key, dataBuffer); + protected void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer, int loopId) + { + tablePipelineManager.routeRecord(key, new Pair<>(dataBuffer, loopId)); } @Override - public boolean isRunning() { + public boolean isRunning() + { return running.get(); } @Override - public void stopProcessor() { + public void stopProcessor() + { running.set(false); transactionProviderThread.interrupt(); transactionProcessorThread.interrupt(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java index c20df77..c62887a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java @@ -193,8 +193,8 @@ private void consumeQueue(SchemaTableName key, BlockingQueue handleRowChangeSourceRecord(key, valueBuffer); - case TRANS -> handleTransactionSourceRecord(valueBuffer); + case ROW -> handleRowChangeSourceRecord(0, valueBuffer, 0); + case TRANS -> handleTransactionSourceRecord(valueBuffer, 0); } } } catch (InterruptedException e) diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java index 061e3d3..4443c80 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java @@ -98,6 +98,11 @@ public static FlushRateLimiter getInstance() return instance; } + public static FlushRateLimiter getNewInstance() + { + return new FlushRateLimiter(); + } + public void acquire(int num) { if (enableRateLimiter && semaphore != null) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java b/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java new file mode 100644 index 0000000..b8f1c2c --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java @@ -0,0 +1,37 @@ +package io.pixelsdb.pixels.sink.util; + +/** + * Inner class to hold and manage per-table transaction row counts. + */ +public class TableCounters +{ + // currentCount is volatile for visibility across threads, as it's incremented during writeRow. + private volatile int currentCount = 0; + private final int totalCount; // The expected total number of rows + + public TableCounters(int totalCount) + { + this.totalCount = totalCount; + } + + public void increment() + { + currentCount++; + } + + public boolean isComplete() + { + // Checks if the processed count meets or exceeds the expected total count. + return currentCount >= totalCount; + } + + public int getCurrentCount() + { + return currentCount; + } + + public int getTotalCount() + { + return totalCount; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index a43242f..29e7f00 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -21,18 +21,126 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.pixelsdb.pixels.sink.util.TableCounters; +import lombok.Getter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +/** + * NoneWriter implementation used for testing and metrics collection. + * It tracks transaction completeness based on row counts provided in the TXEND metadata, + * ensuring robust handling of out-of-order and concurrent TX BEGIN, TX END, and ROWChange events. + */ public class NoneWriter implements PixelsSinkWriter { + private static final Logger LOGGER = LoggerFactory.getLogger(NoneWriter.class); + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + /** + * Helper class to manage the state of a single transaction, decoupling the row accumulation + * from the final TableCounters initialization (which requires total counts from TX END). + */ + public static class TransactionContext + { + @Getter + private volatile boolean endReceived = false; + + // Key: Full Table Name + private Map tableCounters = null; + + // Key: Full Table Name, Value: Row Count + private final Map preEndCounts = new ConcurrentHashMap<>(); + + public void setEndReceived(Map counters) { + this.tableCounters = counters; + this.endReceived = true; + } + + /** + * @param table Full table name + */ + public void incrementPreEndCount(String table) { + preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); + } + } + + + /** + * Data structure to track transaction progress: + * Map + */ + private final Map transTracker = new ConcurrentHashMap<>(); + + + /** + * Checks if all tables within a transaction have reached their expected row count. + * If complete, the transaction is removed from the tracker and final metrics are recorded. + * + * @param transId The ID of the transaction to check. + */ + private void checkAndCleanupTransaction(String transId) + { + TransactionContext context = transTracker.get(transId); + + if (context == null || !context.isEndReceived()) + { + // Transaction has not received TX END or has been cleaned up already. + return; + } + + Map tableMap = context.tableCounters; + if (tableMap == null || tableMap.isEmpty()) + { + // Empty transaction with no tables. Clean up immediately. + transTracker.remove(transId); + metricsFacade.recordTransaction(); + metricsFacade.recordTransactionRowCount(0); + LOGGER.info("Transaction {} (empty) successfully completed and removed from tracker.", transId); + return; + } + + boolean allComplete = true; + int actualProcessedRows = 0; + + // Iterate through all tables to check completion status + for (Map.Entry entry : tableMap.entrySet()) + { + TableCounters counters = entry.getValue(); + actualProcessedRows += counters.getCurrentCount(); + if (!counters.isComplete()) + { + allComplete = false; + } + } + + if (allComplete) + { + // All rows expected have been processed. Remove and record metrics. + transTracker.remove(transId); + LOGGER.info("Transaction {} successfully completed and removed from tracker. Total rows: {}.", transId, actualProcessedRows); + + // Record final transaction metrics only upon completion + metricsFacade.recordTransaction(); + metricsFacade.recordTransactionRowCount(actualProcessedRows); + } else + { + // Not complete, keep tracking + LOGGER.debug("Transaction {} is partially complete ({} rows processed). Keeping tracker entry.", transId, actualProcessedRows); + } + } + + // --- Interface Methods --- @Override public void flush() { - + // No-op for NoneWriter } @Override @@ -40,13 +148,47 @@ public boolean writeRow(RowChangeEvent rowChangeEvent) { metricsFacade.recordRowEvent(); metricsFacade.recordRowChange(rowChangeEvent.getTable(), rowChangeEvent.getOp()); + try { rowChangeEvent.initIndexKey(); metricsFacade.recordPrimaryKeyUpdateDistribution(rowChangeEvent.getTable(), rowChangeEvent.getAfterKey().getKey()); + + // Get transaction ID and table name + String transId = rowChangeEvent.getTransaction().getId(); + String fullTable = rowChangeEvent.getFullTableName(); + + // 1. Get or create the transaction context + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + + // 2. Check if TX END has arrived + if (context.isEndReceived()) + { + // TX END arrived: Use official TableCounters + TableCounters counters = context.tableCounters.get(fullTable); + if (counters != null) + { + // Increment the processed row count for this table + counters.increment(); + + // If this table completed, check if the entire transaction is complete. + if (counters.isComplete()) + { + checkAndCleanupTransaction(transId); + } + } else + { + LOGGER.warn("Row received for TransId {} / Table {} but was not included in TX END metadata.", transId, fullTable); + } + } else + { + // TX END has not arrived: Accumulate count in preEndCounts + context.incrementPreEndCount(fullTable); + LOGGER.debug("Row received for TransId {} / Table {} before TX END. Accumulating count.", transId, fullTable); + } } catch (SinkException e) { - throw new RuntimeException(e); + throw new RuntimeException("Error processing row key or metrics.", e); } return true; @@ -55,15 +197,64 @@ public boolean writeRow(RowChangeEvent rowChangeEvent) @Override public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { - if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) + String transId = transactionMetadata.getId(); + + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) { - metricsFacade.recordTransaction(); - int cnt = 0; + // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). + transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + LOGGER.debug("Transaction {} BEGIN received.", transId); + + } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) + { + // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. + + // Get existing context or create a new one (in case BEGIN was missed). + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + + // --- Initialization Step: Set Total Counts --- + Map newTableCounters = new ConcurrentHashMap<>(); for(SinkProto.DataCollection dataCollection: transactionMetadata.getDataCollectionsList()) { - cnt += dataCollection.getEventCount(); + String fullTable = dataCollection.getDataCollection(); + // Create official counter with total count + newTableCounters.put(fullTable, new TableCounters((int)dataCollection.getEventCount())); + } + + // Set the final state (must be volatile write) + context.setEndReceived(newTableCounters); + + // --- Merge Step: Apply pre-received rows --- + for (Map.Entry preEntry : context.preEndCounts.entrySet()) + { + String table = preEntry.getKey(); + int accumulatedCount = preEntry.getValue().get(); + TableCounters finalCounter = newTableCounters.get(table); + + if (finalCounter != null) + { + // Apply the accumulated count to the official counter + for(int i = 0; i < accumulatedCount; i++) + { + finalCounter.increment(); + } + } else + { + LOGGER.warn("Pre-received rows for table {} (count: {}) but table was not in TX END metadata. Discarding accumulated count.", table, accumulatedCount); + } + } + + if (!newTableCounters.isEmpty()) + { + LOGGER.debug("Transaction {} END received. Tracking initialized and pre-counts merged for {} tables.", transId, newTableCounters.size()); + } else + { + LOGGER.info("Transaction {} END received with zero row collections. Marking as complete.", transId); } - metricsFacade.recordTransactionRowCount(cnt); + + // --- Cleanup/Validation Step --- + // Trigger cleanup. This will validate if all rows (pre and post END) have satisfied the total counts. + checkAndCleanupTransaction(transId); } return true; } @@ -71,6 +262,30 @@ public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) @Override public void close() throws IOException { + // No-op for NoneWriter + LOGGER.info("Remaining unfinished transactions on close: {}", transTracker.size()); + // Log details of transactions that were never completed + if (!transTracker.isEmpty()) + { + transTracker.forEach((transId, context) -> + { + if (context.isEndReceived() && context.tableCounters != null) + { + context.tableCounters.forEach((table, counters) -> + { + if (!counters.isComplete()) + { + LOGGER.warn("Unfinished transaction {}: Table {} - Processed {}/{} rows.", + transId, table, counters.getCurrentCount(), counters.getTotalCount()); + } + }); + } else + { + LOGGER.warn("Unfinished transaction {}: TX END not received. Pre-received rows: {}", + transId, context.preEndCounts); + } + }); + } } -} +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java index 21bbbf8..27bccab 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java @@ -26,12 +26,19 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; +import io.pixelsdb.pixels.sink.util.TableCounters; +import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; /** * @package: io.pixelsdb.pixels.sink.writer @@ -45,6 +52,91 @@ public class ProtoWriter implements PixelsSinkWriter private final RotatingWriterManager writerManager; private final TableMetadataRegistry instance; + + private static class TransactionContext + { + @Getter + private volatile boolean endReceived = false; + + // Key: Full Table Name + private Map tableCounters = null; + + // Key: Full Table Name, Value: Row Count + private final Map preEndCounts = new ConcurrentHashMap<>(); + + public void setEndReceived(Map counters) + { + this.tableCounters = counters; + this.endReceived = true; + } + + public List rowChangeEventList = new ArrayList<>(); + public SinkProto.TransactionMetadata txBegin; + public SinkProto.TransactionMetadata txEnd; + + /** + * @param table Full table name + */ + public void incrementPreEndCount(String table) { + preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); + } + } + + + /** + * Data structure to track transaction progress: + * Map + */ + private final Map transTracker = new ConcurrentHashMap<>(); + + + /** + * Checks if all tables within a transaction have reached their expected row count. + * If complete, the transaction is removed from the tracker and final metrics are recorded. + * + * @param transId The ID of the transaction to check. + */ + private void checkAndCleanupTransaction(String transId) + { + TransactionContext context = transTracker.get(transId); + + if (context == null || !context.isEndReceived()) + { + // Transaction has not received TX END or has been cleaned up already. + return; + } + + Map tableMap = context.tableCounters; + if (tableMap == null || tableMap.isEmpty()) + { + // Empty transaction with no tables. Clean up immediately. + transTracker.remove(transId); + LOGGER.info("Transaction {} (empty) successfully completed and removed from tracker.", transId); + return; + } + + boolean allComplete = true; + int actualProcessedRows = 0; + + // Iterate through all tables to check completion status + for (Map.Entry entry : tableMap.entrySet()) + { + TableCounters counters = entry.getValue(); + if (!counters.isComplete()) + { + allComplete = false; + } + } + + if (allComplete) + { + transTracker.remove(transId); + ByteBuffer transInfo = getTransBuffer(context); + transInfo.rewind(); + writeBuffer(transInfo); + } + } + public ProtoWriter() throws IOException { PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); @@ -57,15 +149,92 @@ public ProtoWriter() throws IOException @Override public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { + String transId = transactionMetadata.getId(); + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) + { + // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). + TransactionContext transactionContext = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + LOGGER.debug("Transaction {} BEGIN received.", transId); + transactionContext.txBegin = transactionMetadata; + } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) + { + // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. - byte[] transData = transactionMetadata.toByteArray(); - return writeData(-1, transData); - // ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); -// buffer.putInt(ProtoType.TRANS.toInt()); -// return writeData(buffer.array(), transData); - } + // Get existing context or create a new one (in case BEGIN was missed). + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + + // --- Initialization Step: Set Total Counts --- + Map newTableCounters = new ConcurrentHashMap<>(); + for(SinkProto.DataCollection dataCollection: transactionMetadata.getDataCollectionsList()) + { + String fullTable = dataCollection.getDataCollection(); + // Create official counter with total count + newTableCounters.put(fullTable, new TableCounters((int)dataCollection.getEventCount())); + } + + // Set the final state (must be volatile write) + context.setEndReceived(newTableCounters); + + // --- Merge Step: Apply pre-received rows --- + for (Map.Entry preEntry : context.preEndCounts.entrySet()) + { + String table = preEntry.getKey(); + int accumulatedCount = preEntry.getValue().get(); + TableCounters finalCounter = newTableCounters.get(table); - public boolean write(SinkProto.RowRecord rowRecord) + if (finalCounter != null) + { + // Apply the accumulated count to the official counter + for(int i = 0; i < accumulatedCount; i++) + { + finalCounter.increment(); + } + } else + { + LOGGER.warn("Pre-received rows for table {} (count: {}) but table was not in TX END metadata. Discarding accumulated count.", table, accumulatedCount); + } + } + context.txEnd = transactionMetadata; + + // --- Cleanup/Validation Step --- + // Trigger cleanup. This will validate if all rows (pre and post END) have satisfied the total counts. + checkAndCleanupTransaction(transId); + } + return true; + } + private ByteBuffer getTransBuffer(TransactionContext transactionContext) + { + int total = 0; + byte[] transDataBegin = transactionContext.txBegin.toByteArray(); + ByteBuffer beginByteBuffer = writeData(-1, transDataBegin); + total += beginByteBuffer.limit(); + beginByteBuffer.rewind(); + byte[] transDataEnd = transactionContext.txEnd.toByteArray(); + ByteBuffer endByteBuffer = writeData(-1, transDataEnd); + endByteBuffer.rewind(); + total += endByteBuffer.limit(); + List rowEvents = new ArrayList<>(); + for(RowChangeEvent rowChangeEvent: transactionContext.rowChangeEventList) + { + ByteBuffer byteBuffer = write(rowChangeEvent.getRowRecord()); + if(byteBuffer == null) + { + return null; + } + byteBuffer.rewind(); + rowEvents.add(byteBuffer); + total += byteBuffer.limit(); + } + ByteBuffer buffer = ByteBuffer.allocate(total); + buffer.put(beginByteBuffer.array()); + for(ByteBuffer rowEvent: rowEvents) + { + buffer.put(rowEvent.array()); + } + buffer.put(endByteBuffer.array()); + return buffer; + } + public ByteBuffer write(SinkProto.RowRecord rowRecord) { byte[] rowData = rowRecord.toByteArray(); String tableName = rowRecord.getSource().getTable(); @@ -78,40 +247,19 @@ public boolean write(SinkProto.RowRecord rowRecord) } catch (SinkException e) { LOGGER.error("Error while getting schema table id.", e); - return false; + return null; } { return writeData((int) tableId, rowData); } - -// ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES + Long.BYTES); -// keyBuffer.putInt(ProtoType.ROW.toInt()) -// .putLong(tableId); - - -// byte[] schemaNameBytes = schemaName.getBytes(); -// byte[] tableNameBytes = tableName.getBytes(); -// -// ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES * 3 + schemaNameBytes.length + tableNameBytes.length); -// keyBuffer.putInt(ProtoType.ROW.toInt()).putInt(schemaNameBytes.length).putInt(tableNameBytes.length); -// keyBuffer.put(schemaNameBytes).put(tableNameBytes); -// return writeData(keyBuffer.array(), rowData); } // key: -1 means transaction, else means table id - private boolean writeData(int key, byte[] data) + private ByteBuffer writeData(int key, byte[] data) { ByteBuffer buf = ByteBuffer.allocate(Integer.BYTES + Integer.BYTES + data.length).order(ByteOrder.BIG_ENDIAN); // key + value len + data buf.putInt(key).putInt(data.length).put(data); - return writeBuffer(buf); - } - - private boolean writeData(byte[] key, byte[] data) - { - ByteBuffer buf = ByteBuffer.allocate(Integer.BYTES + Integer.BYTES + key.length + data.length).order(ByteOrder.BIG_ENDIAN); // rowLen + type + data - - buf.putInt(key.length).putInt(data.length).put(key).put(data); - return writeBuffer(buf); + return buf; } private synchronized boolean writeBuffer(ByteBuffer buf) @@ -133,7 +281,37 @@ private synchronized boolean writeBuffer(ByteBuffer buf) @Override public boolean writeRow(RowChangeEvent rowChangeEvent) { - return write(rowChangeEvent.getRowRecord()); + String transId = rowChangeEvent.getTransaction().getId(); + String fullTable = rowChangeEvent.getFullTableName(); + + // 1. Get or create the transaction context + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + context.rowChangeEventList.add(rowChangeEvent); + // 2. Check if TX END has arrived + if (context.isEndReceived()) + { + // TX END arrived: Use official TableCounters + TableCounters counters = context.tableCounters.get(fullTable); + if (counters != null) + { + // Increment the processed row count for this table + counters.increment(); + + // If this table completed, check if the entire transaction is complete. + if (counters.isComplete()) + { + checkAndCleanupTransaction(transId); + } + } else + { + LOGGER.warn("Row received for TransId {} / Table {} but was not included in TX END metadata.", transId, fullTable); + } + } else + { + context.incrementPreEndCount(fullTable); + LOGGER.debug("Row received for TransId {} / Table {} before TX END. Accumulating count.", transId, fullTable); + } + return true; } @Override diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 2dcf7b4..987e4d6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -122,7 +122,7 @@ private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkExc // startTrans(txBegin.getId()).get(); try { - flushRateLimiter.acquire(1); + // flushRateLimiter.acquire(1); startTransSync(txBegin.getId()); } catch (SinkException e) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 42c728b..422cceb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -70,6 +70,7 @@ public class SinkContext private boolean failed = false; @Getter + @Setter private volatile Long startTime = null; public SinkContext(String sourceTxId) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 3392881..2be3f6d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -33,6 +33,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -165,15 +166,20 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) return; } - // ctx.setStartTime(System.currentTimeMillis()); - + ctx.setStartTime(System.currentTimeMillis()); + boolean abort = false; try { try { ctx.tableCounterLock.lock(); + long startTs= System.currentTimeMillis(); while (!ctx.isCompleted(txEnd)) { + if(abort) + { + break; + } LOGGER.debug("TX End Get Lock {}", txId); LOGGER.debug("Waiting for events in TX {}: {}", txId, txEnd.getDataCollectionsList().stream() @@ -181,7 +187,12 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + "/" + dc.getEventCount()) .collect(Collectors.joining(", "))); - ctx.tableCounterCond.await(); + ctx.tableCounterCond.await(10, TimeUnit.SECONDS); + if(System.currentTimeMillis()-startTs> 20 * 1000) + { + abort = true; + LOGGER.warn("Long Transaction {}, took {}", txId, System.currentTimeMillis()-startTs); + } } } finally { @@ -190,7 +201,7 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) removeSinkContext(txId); - boolean failed = ctx.isFailed(); + boolean failed = ctx.isFailed() | abort; if (!failed) { LOGGER.trace("Committed transaction: {}", txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 0cf0ee7..493e9a8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -92,7 +92,12 @@ public void flush() { if (smallBatch != null && !smallBatch.isEmpty()) { - tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); + RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(txId, smallBatch); + if(builder == null) + { + continue; + } + tableUpdateDataBuilderList.add(builder); tableUpdateCount.add(smallBatch.size()); } txIds.add(currTxId); @@ -105,12 +110,15 @@ public void flush() if (smallBatch != null) { - // tableUpdateData.add(buildTableUpdateDataFromBatch(txId, smallBatch).setBucketId(bucketId).setTxId(currentTxId).build()); - tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); - tableUpdateCount.add(smallBatch.size()); + RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(txId, smallBatch); + if(builder != null) + { + tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); + tableUpdateCount.add(smallBatch.size()); + } } - // flushRateLimiter.acquire(batch.size()); + flushRateLimiter.acquire(batch.size()); long txStartTime = System.currentTimeMillis(); // if(freshnessLevel.equals("embed")) @@ -122,10 +130,10 @@ public void flush() } - for(String writeTxId: txIds) - { - sinkContextManager.getSinkContext(writeTxId).setCurrStartTime(); - } +// for(String writeTxId: txIds) +// { +// sinkContextManager.getSinkContext(writeTxId).setCurrStartTime(); +// } List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) @@ -162,7 +170,10 @@ private void failCtxs(List txIds) for (String writeTxId : txIds) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); - sinkContext.setFailed(true); + if(sinkContext != null) + { + sinkContext.setFailed(true); + } } } @@ -185,6 +196,10 @@ private void updateCtxCounters(List txIds, List fullTableName, L private RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(txId); + if(sinkContext == null) + { + return null; + } try { sinkContext.getLock().lock(); From 5c3162b84e1bd50c048989f318a5750d3032c30a Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 3 Dec 2025 07:28:01 +0000 Subject: [PATCH 37/53] Support Single Table Transaction --- conf/pixels-sink.aws.properties | 10 +- .../pixels/sink/event/RowChangeEvent.java | 13 + .../sink/freshness/FreshnessClient.java | 6 +- .../pixels/sink/writer/NoneWriter.java | 237 +++++------------- .../pixels/sink/writer/ProtoWriter.java | 186 +++++++------- .../writer/retina/RetinaServiceProxy.java | 7 +- .../sink/writer/retina/RetinaWriter.java | 39 +-- .../sink/writer/retina/SinkContext.java | 56 ++++- .../writer/retina/SinkContextManager.java | 164 +++++------- .../writer/retina/TableCrossTxWriter.java | 33 ++- .../retina/TableSingleRecordWriter.java | 129 ++++++++++ .../sink/writer/retina/TableWriter.java | 16 +- .../sink/writer/retina/TableWriterProxy.java | 14 +- .../sink/writer/retina/TransactionMode.java | 1 + .../sink/writer/retina/TransactionProxy.java | 77 ++++-- 15 files changed, 542 insertions(+), 446 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 26298a8..e977ec5 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -3,7 +3,7 @@ sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet sink.datasource.rate.limit=30000 # Sink Config: retina | csv | proto | none -sink.mode=none +sink.mode=retina sink.retina.client=2 sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_test.csv @@ -13,7 +13,7 @@ trino.user=pixels trino.password=password # row or txn or embed -sink.monitor.freshness.level=txn +sink.monitor.freshness.level=embed sink.storage.loop=false # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -47,7 +47,7 @@ sink.commit.batch.worker=32 sink.commit.batch.delay=1000 ## Proto Config sink.proto.dir=file:///home/ubuntu/disk1/hybench -sink.proto.data=hybench10_7 +sink.proto.data=hybench10_10 sink.proto.maxRecords=1000000 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 @@ -56,8 +56,8 @@ transaction.topic.suffix=transaction #transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 -## Batch or trans or record -sink.trans.mode=batch +## batch or single or record +sink.trans.mode=record # Sink Metrics sink.monitor.enable=true sink.monitor.port=9465 diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 6836285..0f92800 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -144,6 +144,19 @@ public void initIndexKey() throws SinkException indexKeyInited = true; } + public void updateIndexKey() throws SinkException + { + if (hasBeforeData()) + { + this.beforeKey = generateIndexKey(tableMetadata, beforeValueMap); + } + + if (hasAfterData()) + { + this.afterKey = generateIndexKey(tableMetadata, afterValueMap); + } + } + public int getBeforeBucketFromIndex() { assert indexKeyInited; diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 8f142a6..2bddb04 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -22,12 +22,14 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.util.DateUtil; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.*; import java.util.*; +import java.util.Date; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -230,10 +232,10 @@ void queryAndCalculateFreshness() { // Timestamp when the query is sent (t_send) long tSendMillis = System.currentTimeMillis(); - + String tSendMillisStr = DateUtil.convertDateToString(new Date(tSendMillis)); // Query to find the latest timestamp in the table // Assumes 'freshness_ts' is a long-type epoch timestamp (milliseconds) - String query = String.format("SELECT max(freshness_ts) FROM %s", tableName); + String query = String.format("SELECT max(freshness_ts) FROM %s WHERE freshness_ts < TIMESTAMP '%s'", tableName, tSendMillisStr); try (Statement statement = connection.createStatement(); ResultSet rs = statement.executeQuery(query)) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index 29e7f00..5268bc5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -22,6 +22,7 @@ import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.pixelsdb.pixels.sink.util.TableCounters; +import io.pixelsdb.pixels.sink.writer.retina.SinkContext; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,6 +31,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; /** * NoneWriter implementation used for testing and metrics collection. @@ -42,42 +44,12 @@ public class NoneWriter implements PixelsSinkWriter private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - /** - * Helper class to manage the state of a single transaction, decoupling the row accumulation - * from the final TableCounters initialization (which requires total counts from TX END). - */ - public static class TransactionContext - { - @Getter - private volatile boolean endReceived = false; - - // Key: Full Table Name - private Map tableCounters = null; - - // Key: Full Table Name, Value: Row Count - private final Map preEndCounts = new ConcurrentHashMap<>(); - - public void setEndReceived(Map counters) { - this.tableCounters = counters; - this.endReceived = true; - } - - /** - * @param table Full table name - */ - public void incrementPreEndCount(String table) { - preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); - } - } - - /** * Data structure to track transaction progress: * Map */ private final Map transTracker = new ConcurrentHashMap<>(); - /** * Checks if all tables within a transaction have reached their expected row count. * If complete, the transaction is removed from the tracker and final metrics are recorded. @@ -88,42 +60,19 @@ private void checkAndCleanupTransaction(String transId) { TransactionContext context = transTracker.get(transId); - if (context == null || !context.isEndReceived()) + if (context == null) { - // Transaction has not received TX END or has been cleaned up already. return; } - Map tableMap = context.tableCounters; - if (tableMap == null || tableMap.isEmpty()) - { - // Empty transaction with no tables. Clean up immediately. - transTracker.remove(transId); - metricsFacade.recordTransaction(); - metricsFacade.recordTransactionRowCount(0); - LOGGER.info("Transaction {} (empty) successfully completed and removed from tracker.", transId); - return; - } - - boolean allComplete = true; - int actualProcessedRows = 0; - - // Iterate through all tables to check completion status - for (Map.Entry entry : tableMap.entrySet()) - { - TableCounters counters = entry.getValue(); - actualProcessedRows += counters.getCurrentCount(); - if (!counters.isComplete()) - { - allComplete = false; - } - } + boolean allComplete = context.sinkContext.isCompleted(); + int actualProcessedRows = context.sinkContext.getProcessedRowsNum(); if (allComplete) { // All rows expected have been processed. Remove and record metrics. transTracker.remove(transId); - LOGGER.info("Transaction {} successfully completed and removed from tracker. Total rows: {}.", transId, actualProcessedRows); + LOGGER.trace("Transaction {} successfully completed and removed from tracker. Total rows: {}.", transId, actualProcessedRows); // Record final transaction metrics only upon completion metricsFacade.recordTransaction(); @@ -135,128 +84,65 @@ private void checkAndCleanupTransaction(String transId) } } - // --- Interface Methods --- - @Override public void flush() { // No-op for NoneWriter } + // --- Interface Methods --- + @Override public boolean writeRow(RowChangeEvent rowChangeEvent) { - metricsFacade.recordRowEvent(); - metricsFacade.recordRowChange(rowChangeEvent.getTable(), rowChangeEvent.getOp()); - - try - { - rowChangeEvent.initIndexKey(); - metricsFacade.recordPrimaryKeyUpdateDistribution(rowChangeEvent.getTable(), rowChangeEvent.getAfterKey().getKey()); + metricsFacade.recordRowEvent(); + metricsFacade.recordRowChange(rowChangeEvent.getTable(), rowChangeEvent.getOp()); + try + { + rowChangeEvent.initIndexKey(); + metricsFacade.recordPrimaryKeyUpdateDistribution(rowChangeEvent.getTable(), rowChangeEvent.getAfterKey().getKey()); - // Get transaction ID and table name - String transId = rowChangeEvent.getTransaction().getId(); - String fullTable = rowChangeEvent.getFullTableName(); + // Get transaction ID and table name + String transId = rowChangeEvent.getTransaction().getId(); + String fullTable = rowChangeEvent.getFullTableName(); - // 1. Get or create the transaction context - TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + // 1. Get or create the transaction context + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); - // 2. Check if TX END has arrived - if (context.isEndReceived()) + context.sinkContext.getTableCounterLock().lock(); + context.incrementEndCount(fullTable); + checkAndCleanupTransaction(transId); + context.sinkContext.getTableCounterLock().unlock(); + } catch (SinkException e) { - // TX END arrived: Use official TableCounters - TableCounters counters = context.tableCounters.get(fullTable); - if (counters != null) - { - // Increment the processed row count for this table - counters.increment(); - - // If this table completed, check if the entire transaction is complete. - if (counters.isComplete()) - { - checkAndCleanupTransaction(transId); - } - } else - { - LOGGER.warn("Row received for TransId {} / Table {} but was not included in TX END metadata.", transId, fullTable); - } - } else - { - // TX END has not arrived: Accumulate count in preEndCounts - context.incrementPreEndCount(fullTable); - LOGGER.debug("Row received for TransId {} / Table {} before TX END. Accumulating count.", transId, fullTable); + throw new RuntimeException("Error processing row key or metrics.", e); } - } catch (SinkException e) - { - throw new RuntimeException("Error processing row key or metrics.", e); - } - - return true; + return true; } @Override public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { - String transId = transactionMetadata.getId(); - - if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) - { - // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). - transTracker.computeIfAbsent(transId, k -> new TransactionContext()); - LOGGER.debug("Transaction {} BEGIN received.", transId); - - } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) - { - // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. + String transId = transactionMetadata.getId(); - // Get existing context or create a new one (in case BEGIN was missed). - TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); - - // --- Initialization Step: Set Total Counts --- - Map newTableCounters = new ConcurrentHashMap<>(); - for(SinkProto.DataCollection dataCollection: transactionMetadata.getDataCollectionsList()) + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) { - String fullTable = dataCollection.getDataCollection(); - // Create official counter with total count - newTableCounters.put(fullTable, new TableCounters((int)dataCollection.getEventCount())); - } + // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). + transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); + LOGGER.debug("Transaction {} BEGIN received.", transId); - // Set the final state (must be volatile write) - context.setEndReceived(newTableCounters); - - // --- Merge Step: Apply pre-received rows --- - for (Map.Entry preEntry : context.preEndCounts.entrySet()) + } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) { - String table = preEntry.getKey(); - int accumulatedCount = preEntry.getValue().get(); - TableCounters finalCounter = newTableCounters.get(table); - - if (finalCounter != null) - { - // Apply the accumulated count to the official counter - for(int i = 0; i < accumulatedCount; i++) - { - finalCounter.increment(); - } - } else - { - LOGGER.warn("Pre-received rows for table {} (count: {}) but table was not in TX END metadata. Discarding accumulated count.", table, accumulatedCount); - } + // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. + + // Get existing context or create a new one (in case BEGIN was missed). + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); + context.sinkContext.getTableCounterLock().lock(); + context.sinkContext.setEndTx(transactionMetadata); + checkAndCleanupTransaction(transId); + context.sinkContext.getTableCounterLock().unlock(); } - - if (!newTableCounters.isEmpty()) - { - LOGGER.debug("Transaction {} END received. Tracking initialized and pre-counts merged for {} tables.", transId, newTableCounters.size()); - } else - { - LOGGER.info("Transaction {} END received with zero row collections. Marking as complete.", transId); - } - - // --- Cleanup/Validation Step --- - // Trigger cleanup. This will validate if all rows (pre and post END) have satisfied the total counts. - checkAndCleanupTransaction(transId); - } - return true; + return true; } @Override @@ -270,22 +156,33 @@ public void close() throws IOException { transTracker.forEach((transId, context) -> { - if (context.isEndReceived() && context.tableCounters != null) - { - context.tableCounters.forEach((table, counters) -> - { - if (!counters.isComplete()) - { - LOGGER.warn("Unfinished transaction {}: Table {} - Processed {}/{} rows.", - transId, table, counters.getCurrentCount(), counters.getTotalCount()); - } - }); - } else - { - LOGGER.warn("Unfinished transaction {}: TX END not received. Pre-received rows: {}", - transId, context.preEndCounts); - } + LOGGER.warn("Unfinished transaction {}",transId); }); } } + + /** + * Helper class to manage the state of a single transaction, decoupling the row accumulation + * from the final TableCounters initialization (which requires total counts from TX END). + */ + public static class TransactionContext + { + // Key: Full Table Name, Value: Row Count + private SinkContext sinkContext = null; + + + TransactionContext(String txId) + { + this.sinkContext = new SinkContext(txId); + } + + + /** + * @param table Full table name + */ + public void incrementEndCount(String table) + { + sinkContext.updateCounter(table, 1); + } + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java index 27bccab..9d6a02f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java @@ -39,6 +39,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; /** * @package: io.pixelsdb.pixels.sink.writer @@ -51,38 +52,7 @@ public class ProtoWriter implements PixelsSinkWriter private final Logger LOGGER = LoggerFactory.getLogger(ProtoWriter.class); private final RotatingWriterManager writerManager; private final TableMetadataRegistry instance; - - - private static class TransactionContext - { - @Getter - private volatile boolean endReceived = false; - - // Key: Full Table Name - private Map tableCounters = null; - - // Key: Full Table Name, Value: Row Count - private final Map preEndCounts = new ConcurrentHashMap<>(); - - public void setEndReceived(Map counters) - { - this.tableCounters = counters; - this.endReceived = true; - } - - public List rowChangeEventList = new ArrayList<>(); - public SinkProto.TransactionMetadata txBegin; - public SinkProto.TransactionMetadata txEnd; - - /** - * @param table Full table name - */ - public void incrementPreEndCount(String table) { - preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); - } - } - - + private final ReentrantLock lock = new ReentrantLock(); /** * Data structure to track transaction progress: * Map @@ -90,6 +60,15 @@ public void incrementPreEndCount(String table) { private final Map transTracker = new ConcurrentHashMap<>(); + public ProtoWriter() throws IOException + { + PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); + + String dataPath = sinkConfig.getSinkProtoData(); + this.writerManager = new RotatingWriterManager(dataPath); + this.instance = TableMetadataRegistry.Instance(); + } + /** * Checks if all tables within a transaction have reached their expected row count. * If complete, the transaction is removed from the tracker and final metrics are recorded. @@ -137,71 +116,70 @@ private void checkAndCleanupTransaction(String transId) } } - public ProtoWriter() throws IOException - { - PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); - - String dataPath = sinkConfig.getSinkProtoData(); - this.writerManager = new RotatingWriterManager(dataPath); - this.instance = TableMetadataRegistry.Instance(); - } - @Override public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { - String transId = transactionMetadata.getId(); - if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) - { - // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). - TransactionContext transactionContext = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); - LOGGER.debug("Transaction {} BEGIN received.", transId); - transactionContext.txBegin = transactionMetadata; - } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) + try { - // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. - - // Get existing context or create a new one (in case BEGIN was missed). - TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); - - // --- Initialization Step: Set Total Counts --- - Map newTableCounters = new ConcurrentHashMap<>(); - for(SinkProto.DataCollection dataCollection: transactionMetadata.getDataCollectionsList()) + lock.lock(); + String transId = transactionMetadata.getId(); + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) { - String fullTable = dataCollection.getDataCollection(); - // Create official counter with total count - newTableCounters.put(fullTable, new TableCounters((int)dataCollection.getEventCount())); - } + // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). + TransactionContext transactionContext = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); + LOGGER.debug("Transaction {} BEGIN received.", transId); + transactionContext.txBegin = transactionMetadata; + } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) + { + // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. - // Set the final state (must be volatile write) - context.setEndReceived(newTableCounters); + // Get existing context or create a new one (in case BEGIN was missed). + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); - // --- Merge Step: Apply pre-received rows --- - for (Map.Entry preEntry : context.preEndCounts.entrySet()) - { - String table = preEntry.getKey(); - int accumulatedCount = preEntry.getValue().get(); - TableCounters finalCounter = newTableCounters.get(table); + // --- Initialization Step: Set Total Counts --- + Map newTableCounters = new ConcurrentHashMap<>(); + for (SinkProto.DataCollection dataCollection : transactionMetadata.getDataCollectionsList()) + { + String fullTable = dataCollection.getDataCollection(); + // Create official counter with total count + newTableCounters.put(fullTable, new TableCounters((int) dataCollection.getEventCount())); + } + + // Set the final state (must be volatile write) + context.setEndReceived(newTableCounters); - if (finalCounter != null) + // --- Merge Step: Apply pre-received rows --- + for (Map.Entry preEntry : context.preEndCounts.entrySet()) { - // Apply the accumulated count to the official counter - for(int i = 0; i < accumulatedCount; i++) + String table = preEntry.getKey(); + int accumulatedCount = preEntry.getValue().get(); + TableCounters finalCounter = newTableCounters.get(table); + + if (finalCounter != null) { - finalCounter.increment(); + // Apply the accumulated count to the official counter + for (int i = 0; i < accumulatedCount; i++) + { + finalCounter.increment(); + } + } else + { + LOGGER.warn("Pre-received rows for table {} (count: {}) but table was not in TX END metadata. Discarding accumulated count.", table, accumulatedCount); } - } else - { - LOGGER.warn("Pre-received rows for table {} (count: {}) but table was not in TX END metadata. Discarding accumulated count.", table, accumulatedCount); } - } - context.txEnd = transactionMetadata; + context.txEnd = transactionMetadata; - // --- Cleanup/Validation Step --- - // Trigger cleanup. This will validate if all rows (pre and post END) have satisfied the total counts. - checkAndCleanupTransaction(transId); + // --- Cleanup/Validation Step --- + // Trigger cleanup. This will validate if all rows (pre and post END) have satisfied the total counts. + checkAndCleanupTransaction(transId); + } + return true; + } finally + { + lock.unlock(); } - return true; } + private ByteBuffer getTransBuffer(TransactionContext transactionContext) { int total = 0; @@ -214,10 +192,10 @@ private ByteBuffer getTransBuffer(TransactionContext transactionContext) endByteBuffer.rewind(); total += endByteBuffer.limit(); List rowEvents = new ArrayList<>(); - for(RowChangeEvent rowChangeEvent: transactionContext.rowChangeEventList) + for (RowChangeEvent rowChangeEvent : transactionContext.rowChangeEventList) { ByteBuffer byteBuffer = write(rowChangeEvent.getRowRecord()); - if(byteBuffer == null) + if (byteBuffer == null) { return null; } @@ -227,13 +205,14 @@ private ByteBuffer getTransBuffer(TransactionContext transactionContext) } ByteBuffer buffer = ByteBuffer.allocate(total); buffer.put(beginByteBuffer.array()); - for(ByteBuffer rowEvent: rowEvents) + for (ByteBuffer rowEvent : rowEvents) { buffer.put(rowEvent.array()); } buffer.put(endByteBuffer.array()); return buffer; } + public ByteBuffer write(SinkProto.RowRecord rowRecord) { byte[] rowData = rowRecord.toByteArray(); @@ -281,6 +260,9 @@ private synchronized boolean writeBuffer(ByteBuffer buf) @Override public boolean writeRow(RowChangeEvent rowChangeEvent) { + try + { + lock.lock(); String transId = rowChangeEvent.getTransaction().getId(); String fullTable = rowChangeEvent.getFullTableName(); @@ -311,7 +293,11 @@ public boolean writeRow(RowChangeEvent rowChangeEvent) context.incrementPreEndCount(fullTable); LOGGER.debug("Row received for TransId {} / Table {} before TX END. Accumulating count.", transId, fullTable); } - return true; + return true; + } finally + { + lock.unlock(); + } } @Override @@ -320,10 +306,36 @@ public void flush() } - @Override public void close() throws IOException { this.writerManager.close(); } + + private static class TransactionContext + { + // Key: Full Table Name, Value: Row Count + private final Map preEndCounts = new ConcurrentHashMap<>(); + public List rowChangeEventList = new ArrayList<>(); + public SinkProto.TransactionMetadata txBegin; + public SinkProto.TransactionMetadata txEnd; + @Getter + private volatile boolean endReceived = false; + // Key: Full Table Name + private Map tableCounters = null; + + public void setEndReceived(Map counters) + { + this.tableCounters = counters; + this.endReceived = true; + } + + /** + * @param table Full table name + */ + public void incrementPreEndCount(String table) + { + preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); + } + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java index cdd0596..216458c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java @@ -18,7 +18,6 @@ package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.RetinaException; -import io.pixelsdb.pixels.common.node.BucketCache; import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.common.utils.RetinaUtils; import io.pixelsdb.pixels.retina.RetinaProto; @@ -49,7 +48,7 @@ public class RetinaServiceProxy public RetinaServiceProxy(int bucketId) { - if(bucketId == -1) + if (bucketId == -1) { this.retinaService = RetinaService.Instance(); } else @@ -58,7 +57,7 @@ public RetinaServiceProxy(int bucketId) } - if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) + if (config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { retinaStream = retinaService.startUpdateStream(); } else @@ -122,7 +121,7 @@ public boolean writeTrans(String schemaName, List t public void close() throws IOException { isClosed.compareAndSet(false, true); - if (config.getTransactionMode() == TransactionMode.BATCH && config.getRetinaWriteMode() == RetinaWriteMode.STREAM) + if (config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { retinaStream.close(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 987e4d6..2de7423 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -17,8 +17,6 @@ package io.pixelsdb.pixels.sink.writer.retina; -import io.pixelsdb.pixels.common.exception.TransException; -import io.pixelsdb.pixels.common.transaction.TransService; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; @@ -27,39 +25,43 @@ import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; -import io.prometheus.client.Summary; import org.apache.commons.lang3.RandomUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; public class RetinaWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriter.class); final ExecutorService dispatchExecutor = Executors.newCachedThreadPool(); - private final ExecutorService transactionExecutor = Executors.newFixedThreadPool(2048); private final ScheduledExecutorService timeoutScheduler = Executors.newSingleThreadScheduledExecutor(); private final FlushRateLimiter flushRateLimiter; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final SinkContextManager sinkContextManager; + private final TransactionMode transactionMode; public RetinaWriter() { + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.sinkContextManager = SinkContextManager.getInstance(); this.flushRateLimiter = FlushRateLimiter.getInstance(); + this.transactionMode = config.getTransactionMode(); } @Override public boolean writeTrans(SinkProto.TransactionMetadata txMeta) { + if(transactionMode.equals(TransactionMode.RECORD)) + { + return true; + } + try { if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) @@ -100,13 +102,18 @@ public boolean writeRow(RowChangeEvent event) long collectionOrder = event.getTransaction().getDataCollectionOrder(); long totalOrder = event.getTransaction().getTotalOrder(); - - AtomicBoolean canWrite = new AtomicBoolean(false); - SinkContext ctx = sinkContextManager.getActiveTxContext(event, canWrite); - - if (canWrite.get()) + if(transactionMode.equals(TransactionMode.RECORD)) + { + sinkContextManager.writeRowChangeEvent(null, event); + } else { - sinkContextManager.writeRowChangeEvent(ctx, event); + AtomicBoolean canWrite = new AtomicBoolean(false); + SinkContext ctx = sinkContextManager.getActiveTxContext(event, canWrite); + + if (canWrite.get()) + { + sinkContextManager.writeRowChangeEvent(ctx, event); + } } } catch (SinkException e) { @@ -138,11 +145,7 @@ private void startTransSync(String sourceTxId) throws SinkException private void handleTxEnd(SinkProto.TransactionMetadata txEnd) { - transactionExecutor.submit(() -> - { - sinkContextManager.processTxCommit(txEnd); - } - ); + sinkContextManager.processTxCommit(txEnd); } private void handleNonTxEvent(RowChangeEvent event) throws SinkException @@ -150,7 +153,7 @@ private void handleNonTxEvent(RowChangeEvent event) throws SinkException // virtual tx String randomId = Long.toString(System.currentTimeMillis()) + RandomUtils.nextLong(); writeTrans(buildBeginTransactionMetadata(randomId)); - sinkContextManager.writeRowChangeEvent(randomId, event); + sinkContextManager.writeRandomRowChangeEvent(randomId, event); writeTrans(buildEndTransactionMetadata(event.getFullTableName(), randomId)); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 422cceb..81595dd 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -52,15 +52,14 @@ public class SinkContext @Getter final String sourceTxId; @Getter - Map tableCounters = new ConcurrentHashMap<>(); - @Getter final AtomicInteger pendingEvents = new AtomicInteger(0); @Getter final CompletableFuture completionFuture = new CompletableFuture<>(); - @Getter final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); @Getter + Map tableCounters = new ConcurrentHashMap<>(); + @Getter Queue orphanEvent = new ConcurrentLinkedQueue<>(); @Getter @Setter @@ -73,6 +72,9 @@ public class SinkContext @Setter private volatile Long startTime = null; + @Getter + @Setter + SinkProto.TransactionMetadata endTx; public SinkContext(String sourceTxId) { this.sourceTxId = sourceTxId; @@ -100,20 +102,48 @@ public void updateCounter(String table, long count) tableCounterLock.unlock(); } - boolean isCompleted(SinkProto.TransactionMetadata tx) + public boolean isCompleted() { - for (SinkProto.DataCollection dataCollection : tx.getDataCollectionsList()) + try { - // Long targetEventCount = tableCursors.get(dataCollection.getDataCollection()); - Long targetEventCount = tableCounters.get(dataCollection.getDataCollection()); - long target = targetEventCount == null ? 0 : targetEventCount; - LOGGER.debug("TX {}, Table {}, event count {}, tableCursors {}", tx.getId(), dataCollection.getDataCollection(), dataCollection.getEventCount(), target); - if (dataCollection.getEventCount() > target) + tableCounterLock.lock(); + if(endTx == null) { return false; } + for (SinkProto.DataCollection dataCollection : endTx.getDataCollectionsList()) + { + Long targetEventCount = tableCounters.get(dataCollection.getDataCollection()); + long target = targetEventCount == null ? 0 : targetEventCount; + LOGGER.debug("TX {}, Table {}, event count {}, tableCursors {}", endTx.getId(), dataCollection.getDataCollection(), dataCollection.getEventCount(), target); + if (dataCollection.getEventCount() > target) + { + return false; + } + } + return true; + } finally + { + tableCounterLock.unlock(); + } + + } + + public int getProcessedRowsNum() + { + long num = 0; + try + { + tableCounterLock.lock(); + for(Long counter: tableCounters.values()) + { + num += counter; + } + } finally + { + tableCounterLock.unlock(); } - return true; + return (int)num; } public long getTimestamp() @@ -132,14 +162,14 @@ public void bufferOrphanedEvent(RowChangeEvent event) public void setCurrStartTime() { - if(startTime != null) + if (startTime != null) { return; } synchronized (this) { - if(startTime == null) + if (startTime == null) { startTime = System.currentTimeMillis(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 2be3f6d..bac9f03 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -32,36 +32,24 @@ import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; public class SinkContextManager { private static final Logger LOGGER = LoggerFactory.getLogger(SinkContextManager.class); private static final Logger BUCKET_TRACE_LOGGER = LoggerFactory.getLogger("bucket_trace"); - - - + private static volatile SinkContextManager instance; private final BlockingBoundedMap activeTxContexts = new BlockingBoundedMap<>(100000); // private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(10000); private final TransactionProxy transactionProxy = TransactionProxy.Instance(); - private final TransService transService = TransService.Instance(); private final TableWriterProxy tableWriterProxy; private final CommitMethod commitMethod; - private enum CommitMethod - { - Sync, - Async - } - private SinkContextManager() { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.tableWriterProxy = TableWriterProxy.getInstance(); - if(config.getCommitMethod().equals("sync")) + if (config.getCommitMethod().equals("sync")) { this.commitMethod = CommitMethod.Sync; } else @@ -70,12 +58,14 @@ private SinkContextManager() } } - private static volatile SinkContextManager instance; - - public static SinkContextManager getInstance() { - if (instance == null) { - synchronized (SinkContextManager.class) { - if (instance == null) { + public static SinkContextManager getInstance() + { + if (instance == null) + { + synchronized (SinkContextManager.class) + { + if (instance == null) + { instance = new SinkContextManager(); } } @@ -134,7 +124,7 @@ protected void startTransSync(String sourceTxId) oldCtx.getLock().lock(); try { - if(oldCtx.getPixelsTransCtx() != null) + if (oldCtx.getPixelsTransCtx() != null) { LOGGER.warn("Previous tx {} has been released, maybe due to loop process", sourceTxId); oldCtx.tableCounters = new ConcurrentHashMap<>(); @@ -162,100 +152,61 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) SinkContext ctx = getSinkContext(txId); if (ctx == null) { - LOGGER.warn("Sink Context is null"); - return; + throw new RuntimeException("Sink Context is null"); } ctx.setStartTime(System.currentTimeMillis()); - boolean abort = false; try { - try - { - ctx.tableCounterLock.lock(); - long startTs= System.currentTimeMillis(); - while (!ctx.isCompleted(txEnd)) - { - if(abort) - { - break; - } - LOGGER.debug("TX End Get Lock {}", txId); - LOGGER.debug("Waiting for events in TX {}: {}", txId, - txEnd.getDataCollectionsList().stream() - .map(dc -> dc.getDataCollection() + "=" + - ctx.tableCounters.getOrDefault(dc.getDataCollection(), 0L) + - "/" + dc.getEventCount()) - .collect(Collectors.joining(", "))); - ctx.tableCounterCond.await(10, TimeUnit.SECONDS); - if(System.currentTimeMillis()-startTs> 20 * 1000) - { - abort = true; - LOGGER.warn("Long Transaction {}, took {}", txId, System.currentTimeMillis()-startTs); - } - } - } finally + ctx.tableCounterLock.lock(); + ctx.setEndTx(txEnd); + long startTs = System.currentTimeMillis(); + if(ctx.isCompleted()) { - ctx.tableCounterLock.unlock(); + endTransaction(ctx); } + } finally + { + ctx.tableCounterLock.unlock(); + } + } - - removeSinkContext(txId); - boolean failed = ctx.isFailed() | abort; - if (!failed) + void endTransaction(SinkContext ctx) + { + String txId = ctx.getSourceTxId(); + removeSinkContext(txId); + boolean failed = ctx.isFailed(); + if (!failed) + { + LOGGER.trace("Committed transaction: {}", txId); + switch (commitMethod) { - LOGGER.trace("Committed transaction: {}", txId); - switch(commitMethod) + case Sync -> { - case Sync -> - { - transactionProxy.commitTransSync(ctx); - } - case Async -> - { - transactionProxy.commitTransAsync(ctx); - } + transactionProxy.commitTransSync(ctx); } - - } else - { - LOGGER.info("Abort transaction: {}", txId); - CompletableFuture.runAsync(() -> + case Async -> { - try - { - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); - } catch (TransException e) - { - throw new RuntimeException(e); - } - }).whenComplete((v, ex) -> - { - if (ex != null) - { - LOGGER.error("Rollback failed", ex); - } - }); + transactionProxy.commitTransAsync(ctx); + } } - } catch (InterruptedException e) + + } else { - try + LOGGER.info("Abort transaction: {}", txId); + CompletableFuture.runAsync(() -> { - LOGGER.info("Catch Exception, Abort transaction: {}", txId); - transService.rollbackTrans(ctx.getPixelsTransCtx().getTransId(), false); - } catch (TransException ex) + transactionProxy.rollbackTrans(ctx.getPixelsTransCtx()); + }).whenComplete((v, ex) -> { - LOGGER.error("Failed to abort transaction {}", txId); - ex.printStackTrace(); - LOGGER.error(ex.getMessage()); - throw new RuntimeException(ex); - } - LOGGER.error(e.getMessage()); - LOGGER.error("Failed to commit transaction {}", txId, e); + if (ex != null) + { + LOGGER.error("Rollback failed", ex); + } + }); } } - private void handleOrphanEvents(SinkContext ctx) throws SinkException { Queue buffered = ctx.getOrphanEvent(); @@ -272,20 +223,25 @@ private void handleOrphanEvents(SinkContext ctx) throws SinkException protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException { - event.setTimeStamp(ctx.getTimestamp()); + if(ctx != null) + { + event.setTimeStamp(ctx.getTimestamp()); + } event.initIndexKey(); switch (event.getOp()) { case UPDATE -> { - if(!event.isPkChanged()) + if (!event.isPkChanged()) { writeBeforeEvent(ctx, event); } else { TypeDescription typeDescription = event.getSchema(); - ctx.updateCounter(event.getFullTableName(), -1L); - + if(ctx != null) + { + ctx.updateCounter(event.getFullTableName(), -1L); + } SinkProto.RowRecord.Builder deleteBuilder = event.getRowRecord().toBuilder() .clearAfter().setOp(SinkProto.OperationType.DELETE); RowChangeEvent deleteEvent = new RowChangeEvent(deleteBuilder.build(), typeDescription); @@ -343,7 +299,7 @@ protected void removeSinkContext(String txId) activeTxContexts.remove(txId); } - protected void writeRowChangeEvent(String randomId, RowChangeEvent event) throws SinkException + protected void writeRandomRowChangeEvent(String randomId, RowChangeEvent event) throws SinkException { writeRowChangeEvent(getSinkContext(randomId), event); } @@ -352,4 +308,10 @@ public int getActiveTxnsNum() { return activeTxContexts.size(); } + + private enum CommitMethod + { + Sync, + Async + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 493e9a8..6019223 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -19,7 +19,6 @@ package io.pixelsdb.pixels.sink.writer.retina; -import com.google.protobuf.Message; import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; @@ -46,8 +45,9 @@ public class TableCrossTxWriter extends TableWriter @Getter private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); private final int flushBatchSize; - private final ReentrantLock writeLock = new ReentrantLock(); + protected final ReentrantLock writeLock = new ReentrantLock(); private final int bucketId; + public TableCrossTxWriter(String t, int bucketId) { super(t, bucketId); @@ -93,7 +93,7 @@ public void flush() if (smallBatch != null && !smallBatch.isEmpty()) { RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(txId, smallBatch); - if(builder == null) + if (builder == null) { continue; } @@ -111,7 +111,7 @@ public void flush() if (smallBatch != null) { RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(txId, smallBatch); - if(builder != null) + if (builder != null) { tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); tableUpdateCount.add(smallBatch.size()); @@ -122,7 +122,7 @@ public void flush() long txStartTime = System.currentTimeMillis(); // if(freshnessLevel.equals("embed")) - if(true) + if (true) { long freshness_ts = txStartTime * 1000; FreshnessClient.getInstance().addMonitoredTable(tableName); @@ -145,15 +145,15 @@ public void flush() updateRecordResponseCompletableFuture.thenAccept( resp -> { - if(resp.getHeader().getErrorCode() != 0) + if (resp.getHeader().getErrorCode() != 0) { failCtxs(txIds); } else { long txEndTime = System.currentTimeMillis(); - if(freshnessLevel.equals("row")) + if (freshnessLevel.equals("row")) { - metricsFacade.recordFreshness(txEndTime- txStartTime); + metricsFacade.recordFreshness(txEndTime - txStartTime); } updateCtxCounters(txIds, fullTableName, tableUpdateCount); } @@ -170,7 +170,7 @@ private void failCtxs(List txIds) for (String writeTxId : txIds) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); - if(sinkContext != null) + if (sinkContext != null) { sinkContext.setFailed(true); } @@ -185,18 +185,27 @@ private void updateCtxCounters(List txIds, List fullTableName, L metricsFacade.recordRowEvent(tableUpdateCount.get(i)); String writeTxId = txIds.get(i); SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); - if(sinkContext !=null) + + try { + sinkContext.tableCounterLock.lock(); sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); + if(sinkContext.isCompleted()) + { + SinkContextManager.getInstance().endTransaction(sinkContext); + } + } finally + { + sinkContext.tableCounterLock.unlock(); } } writeLock.unlock(); } - private RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) + protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(txId); - if(sinkContext == null) + if (sinkContext == null) { return null; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java new file mode 100644 index 0000000..3bf4702 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java @@ -0,0 +1,129 @@ +package io.pixelsdb.pixels.sink.writer.retina; + +import io.pixelsdb.pixels.common.transaction.TransContext; +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; +import io.pixelsdb.pixels.sink.util.DataTransform; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class TableSingleRecordWriter extends TableCrossTxWriter +{ + private final TransactionProxy transactionProxy; + + public TableSingleRecordWriter(String t, int bucketId) + { + super(t, bucketId); + this.transactionProxy = TransactionProxy.Instance(); + } + + /** + * Flush any buffered events for the current transaction. + */ + public void flush() + { + List batch; + bufferLock.lock(); + try + { + if (buffer.isEmpty()) + { + return; + } + // Swap buffers quickly under lock + batch = buffer; + buffer = new LinkedList<>(); + } finally + { + bufferLock.unlock(); + } + + TransContext pixelsTransContext = transactionProxy.getNewTransContext(); + + writeLock.lock(); + try + { + List tableUpdateDataBuilderList = new LinkedList<>(); + for (RowChangeEvent event : batch) + { + event.setTimeStamp(pixelsTransContext.getTimestamp()); + event.updateIndexKey(); + } + + RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(pixelsTransContext, batch); + if (builder != null) + { + tableUpdateDataBuilderList.add(builder); + } + + flushRateLimiter.acquire(batch.size()); + long txStartTime = System.currentTimeMillis(); + +// if(freshnessLevel.equals("embed")) + if (true) + { + long freshness_ts = txStartTime * 1000; + FreshnessClient.getInstance().addMonitoredTable(tableName); + DataTransform.updateTimeStamp(tableUpdateDataBuilderList, freshness_ts); + } + + List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); + for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) + { + tableUpdateData.add(tableUpdateDataItem.build()); + } + CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); + + updateRecordResponseCompletableFuture.thenAccept( + resp -> + { + if (resp.getHeader().getErrorCode() != 0) + { + transactionProxy.rollbackTrans(pixelsTransContext); + } else + { + metricsFacade.recordRowEvent(batch.size()); + long txEndTime = System.currentTimeMillis(); + if (freshnessLevel.equals("row")) + { + metricsFacade.recordFreshness(txEndTime - txStartTime); + } + transactionProxy.commitTrans(pixelsTransContext); + } + } + ); + } catch (SinkException e) + { + throw new RuntimeException(e); + } finally + { + writeLock.unlock(); + } + } + + protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(TransContext transContext, List smallBatch) + { + RowChangeEvent event1 = smallBatch.get(0); + RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder() + .setTimestamp(transContext.getTimestamp()) + .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) + .setTableName(tableName); + try + { + for (RowChangeEvent smallEvent : smallBatch) + { + addUpdateData(smallEvent, builder); + } + } catch (SinkException e) + { + throw new RuntimeException("Flush failed for table " + tableName, e); + } + return builder; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 9b685ea..fa43cbf 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -51,6 +51,9 @@ public abstract class TableWriter protected final ReentrantLock bufferLock = new ReentrantLock(); protected final String tableName; protected final long flushInterval; + protected final FlushRateLimiter flushRateLimiter; + protected final SinkContextManager sinkContextManager; + protected final String freshnessLevel; // Shared state (protected by lock) protected List buffer = new LinkedList<>(); protected volatile String currentTxId = null; @@ -59,10 +62,7 @@ public abstract class TableWriter protected String fullTableName; protected PixelsSinkConfig config; protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); - protected final FlushRateLimiter flushRateLimiter; - protected final SinkContextManager sinkContextManager; - protected final String freshnessLevel; - + protected TransactionMode transactionMode; protected TableWriter(String tableName, int bucketId) { this.config = PixelsSinkConfigFactory.getInstance(); @@ -72,6 +72,7 @@ protected TableWriter(String tableName, int bucketId) this.sinkContextManager = SinkContextManager.getInstance(); this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); this.delegate = new RetinaServiceProxy(bucketId); + this.transactionMode = config.getTransactionMode(); } /** @@ -118,7 +119,10 @@ public boolean write(RowChangeEvent event, SinkContext ctx) bufferLock.lock(); try { - txId = ctx.getSourceTxId(); + if(!transactionMode.equals(TransactionMode.RECORD)) + { + txId = ctx.getSourceTxId(); + } // If this is a new transaction, flush the old one if (needFlush()) { @@ -148,7 +152,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) bufferLock.lock(); try { - if (txId.equals(currentTxId)) + if (transactionMode.equals(TransactionMode.RECORD) || txId.equals(currentTxId)) { flush(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java index 60b7963..88353f6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java @@ -19,11 +19,9 @@ import io.pixelsdb.pixels.common.node.BucketCache; import io.pixelsdb.pixels.daemon.NodeProto; -import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import java.io.Writer; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -33,8 +31,6 @@ public class TableWriterProxy private final TransactionMode transactionMode; private final int retinaCliNum; - record WriterKey(long tableId, NodeProto.NodeInfo nodeInfo, int cliNo) { } - private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); private TableWriterProxy() @@ -63,10 +59,14 @@ protected TableWriter getTableWriter(String tableName, long tableId, int bucket) { return new TableSingleTxWriter(tableName, bucket); } - case BATCH -> + case BATCH -> { return new TableCrossTxWriter(tableName, bucket); } + case RECORD -> + { + return new TableSingleRecordWriter(tableName, bucket); + } default -> { throw new IllegalArgumentException("Unknown transaction mode: " + transactionMode); @@ -74,4 +74,8 @@ protected TableWriter getTableWriter(String tableName, long tableId, int bucket) } }); } + + record WriterKey(long tableId, NodeProto.NodeInfo nodeInfo, int cliNo) + { + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java index 6d9d082..e424332 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java @@ -20,6 +20,7 @@ public enum TransactionMode { SINGLE, + RECORD, BATCH; public static TransactionMode fromValue(String value) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index 30c729a..e63b602 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Queue; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; /** * This class if for pixels trans service @@ -51,6 +52,8 @@ public class TransactionProxy private final int WORKER_COUNT; private final int MAX_WAIT_MS; + private AtomicInteger beginCount = new AtomicInteger(0); + private AtomicInteger commitCount = new AtomicInteger(0); private TransactionProxy() { @@ -82,11 +85,11 @@ private TransactionProxy() public static TransactionProxy Instance() { - if(instance == null) + if (instance == null) { synchronized (TransactionProxy.class) { - if(instance == null) + if (instance == null) { instance = new TransactionProxy(); } @@ -95,6 +98,14 @@ public static TransactionProxy Instance() return instance; } + public static void staticClose() + { + if (instance != null) + { + instance.close(); + } + } + private void requestTransactions() { try @@ -109,6 +120,17 @@ private void requestTransactions() public TransContext getNewTransContext() { + beginCount.incrementAndGet(); + if(true) + { + try + { + return transService.beginTrans(false); + } catch (TransException e) + { + throw null; + } + } TransContext ctx = transContextQueue.poll(); if (ctx != null) { @@ -137,23 +159,39 @@ public void commitTransAsync(SinkContext transContext) public void commitTransSync(SinkContext transContext) { - try - { - transService.commitTrans(transContext.getPixelsTransCtx().getTransId(), false); - metricsFacade.recordTransaction(); - long txEndTime = System.currentTimeMillis(); + commitTrans(transContext.getPixelsTransCtx()); + metricsFacade.recordTransaction(); + long txEndTime = System.currentTimeMillis(); - if(freshnessLevel.equals("txn")) - { - metricsFacade.recordFreshness(txEndTime- transContext.getStartTime()); - } + if (freshnessLevel.equals("txn")) + { + metricsFacade.recordFreshness(txEndTime - transContext.getStartTime()); } - catch (TransException e) + } + + public void commitTrans(TransContext ctx) + { + commitCount.incrementAndGet(); + try + { + transService.commitTrans(ctx.getTransId(), false); + } catch (TransException e) { LOGGER.error("Batch commit failed: {}", e.getMessage(), e); } } + public void rollbackTrans(TransContext ctx) + { + try + { + transService.rollbackTrans(ctx.getTransId(), false); + } catch (TransException e) + { + LOGGER.error("Rollback transaction failed: {}", e.getMessage(), e); + } + } + private void batchCommitWorker() { List batchTransIds = new ArrayList<>(BATCH_SIZE); @@ -198,11 +236,12 @@ private void batchCommitWorker() metricsFacade.recordTransaction(batchTransIds.size()); long txEndTime = System.currentTimeMillis(); - if(freshnessLevel.equals("txn")) + if (freshnessLevel.equals("txn")) { txStartTimes.forEach( - txStartTime -> { - metricsFacade.recordFreshness(txEndTime- txStartTime); + txStartTime -> + { + metricsFacade.recordFreshness(txEndTime - txStartTime); } ); } @@ -249,12 +288,4 @@ public void close() } } } - - public static void staticClose() - { - if(instance != null) - { - instance.close(); - } - } } From 343cb1bf7d23fe8bbb1895440193508559415596 Mon Sep 17 00:00:00 2001 From: Rolland1944 <145736819+Rolland1944@users.noreply.github.com> Date: Wed, 3 Dec 2025 22:11:48 +0800 Subject: [PATCH 38/53] Implement of FlinkWriter and RPC Server (#14) * add flinkwriter and rpc server * Add Rpc Server * fix: fix bugs to make it runnable * add updateTimeStamp method --------- Co-authored-by: Ubuntu --- conf/pixels-sink.aws.properties | 28 +-- .../pixels/sink/config/PixelsSinkConfig.java | 3 + .../pixels/sink/util/DataTransform.java | 47 ++++ .../pixels/sink/writer/PixelsSinkMode.java | 1 + .../sink/writer/PixelsSinkWriterFactory.java | 4 + .../pixels/sink/writer/ProtoWriter.java | 2 +- .../sink/writer/flink/FlinkPollingWriter.java | 206 ++++++++++++++++ .../flink/PixelsPollingServiceImpl.java | 93 ++++++++ .../sink/writer/flink/PollingRpcServer.java | 90 +++++++ .../pixels/sink/writer/RpcEndToEndTest.java | 224 ++++++++++++++++++ 10 files changed, 682 insertions(+), 16 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java create mode 100644 src/test/java/io/pixelsdb/pixels/sink/writer/RpcEndToEndTest.java diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index e977ec5..c96476d 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,20 +1,16 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=30000 -# Sink Config: retina | csv | proto | none -sink.mode=retina +sink.datasource.rate.limit=150000 +# Sink Config: retina | csv | proto | flink | none +sink.mode=FLINK sink.retina.client=2 -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_test.csv -# trino for freshness query -trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x -trino.user=pixels -trino.password=password - -# row or txn or embed -sink.monitor.freshness.level=embed -sink.storage.loop=false +sink.monitor.report.enable=false +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate150k_6.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness150k_6.csv +# row or txn +sink.monitor.freshness.level=row +sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 group.id=3078 @@ -46,9 +42,11 @@ sink.commit.batch.size=200 sink.commit.batch.worker=32 sink.commit.batch.delay=1000 ## Proto Config -sink.proto.dir=file:///home/ubuntu/disk1/hybench +sink.proto.dir=file:///home/ubuntu/disk1/hybench/ sink.proto.data=hybench10_10 -sink.proto.maxRecords=1000000 +sink.proto.maxRecords=100000 +## Flink Config +sink.flink.server.port=9091 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 9cee533..0eecdc4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -61,6 +61,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.remote.port", defaultValue = "9090") private short remotePort; + @ConfigKey(value = "sink.flink.server.port", defaultValue = "9091") + private int sinkFlinkServerPort; + @ConfigKey(value = "sink.timeout.ms", defaultValue = "30000") private int timeoutMs; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java index a0a14d9..d1daa8d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -22,6 +22,8 @@ import com.google.protobuf.ByteString; import io.pixelsdb.pixels.retina.RetinaProto; + import io.pixelsdb.pixels.sink.SinkProto; + import java.util.ArrayList; import java.nio.ByteBuffer; import java.util.List; @@ -57,4 +59,49 @@ public static void updateTimeStamp(List upd } } } + /** + * 遍历 RowRecord 列表,为每个记录的 'after' 镜像的最后一列更新时间戳。 + * 由于 RowRecord 是不可变的,此方法会返回一个包含已修改记录的新列表。 + * + * @param records 原始的 RowRecord 列表。 + * @param timestamp 要设置的时间戳 (long 类型)。 + * @return 包含更新后时间戳的 RowRecord 新列表。 + */ + public static List updateRecordTimestamp(List records, long timestamp) { + // 处理空或 null 列表的边界情况 + if (records == null || records.isEmpty()) { + return records; + } + // 1. 一次性将 long 转换为 ByteString,提高效率 + ByteString timestampBytes = longToByteString(timestamp); + SinkProto.ColumnValue timestampColumn = SinkProto.ColumnValue.newBuilder().setValue(timestampBytes).build(); + // 2. 创建一个新列表来存储修改后的记录 + List updatedRecords = new ArrayList<>(records.size()); + // 3. 遍历所有记录 + for (SinkProto.RowRecord record : records) { + SinkProto.RowRecord.Builder recordBuilder = record.toBuilder(); + // 4. 只处理包含 'after' 镜像的操作类型 (INSERT, UPDATE, SNAPSHOT) + switch (record.getOp()) { + case INSERT: + case UPDATE: + case SNAPSHOT: + if (recordBuilder.hasAfter()) { + SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); + int colCount = afterBuilder.getValuesCount(); + if (colCount > 0) { + // 5. 设置最后一列的值 + afterBuilder.setValues(colCount - 1, timestampColumn); + } + } + break; + case DELETE: + default: + // 对于 DELETE 或其他未知操作,我们不修改记录 + break; + } + // 6. 将构建好的记录(无论是否修改)添加到新列表中 + updatedRecords.add(recordBuilder.build()); + } + return updatedRecords; + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java index 8e6af20..af9962c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java @@ -23,6 +23,7 @@ public enum PixelsSinkMode CSV, RETINA, PROTO, + FLINK, NONE; public static PixelsSinkMode fromValue(String value) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java index 2801cef..2443444 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java @@ -20,6 +20,7 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.writer.retina.RetinaWriter; +import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; import java.io.IOException; @@ -51,6 +52,9 @@ static public PixelsSinkWriter getWriter() case PROTO: writer = new ProtoWriter(); break; + case FLINK: + writer = new FlinkPollingWriter(); + break; case NONE: writer = new NoneWriter(); break; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java index 9d6a02f..fcd2c42 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java @@ -338,4 +338,4 @@ public void incrementPreEndCount(String table) preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); } } -} +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java new file mode 100644 index 0000000..07d3ef9 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java @@ -0,0 +1,206 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package io.pixelsdb.pixels.sink.writer.flink; + +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.util.FlushRateLimiter; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +/** + * FlinkPollingWriter is a PixelsSinkWriter implementation designed for a long-polling pattern. + * It maintains in-memory blocking queues per table, acting as a buffer between the upstream + * data source (producer) and the gRPC service (consumer). + * This class is thread-safe and integrates FlushRateLimiter to control ingress traffic. + * It also manages the lifecycle of the gRPC server. + */ +public class FlinkPollingWriter implements PixelsSinkWriter { + + private static final Logger LOGGER = LoggerFactory.getLogger(FlinkPollingWriter.class); + + // Core data structure: A thread-safe map from table name to a thread-safe blocking queue. + private final Map> tableQueues; + + // Ingress rate limiter to control the data writing speed. + private final FlushRateLimiter rateLimiter; + + // The gRPC server instance managed by this writer. + private final PollingRpcServer pollingRpcServer; + + /** + * Constructor for FlinkPollingWriter. + * Initializes the data structures, rate limiter, and starts the gRPC server. + */ + public FlinkPollingWriter() { + this.tableQueues = new ConcurrentHashMap<>(); + // Get the global RateLimiter instance + this.rateLimiter = FlushRateLimiter.getInstance(); + LOGGER.info("FlinkPollingWriter initialized with FlushRateLimiter."); + + // --- START: New logic to initialize and start the gRPC server --- + try { + // 1. Get configuration + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + int rpcPort = config.getSinkFlinkServerPort(); + // 2. Create the gRPC service implementation first, passing a reference to this writer. + PixelsPollingServiceImpl service = new PixelsPollingServiceImpl(this); + + // 3. Create the PollingRpcServer instance with the service and port. + LOGGER.info("Attempting to start gRPC Polling Server on port {}...", rpcPort); + this.pollingRpcServer = new PollingRpcServer(service, rpcPort); + // 4. Start the server. + this.pollingRpcServer.start(); + LOGGER.info("gRPC Polling Server successfully started and is managed by FlinkPollingWriter."); + } catch (IOException e) { + // If the server fails to start, the writer cannot function. + // Throw a RuntimeException to fail the Flink task initialization. + LOGGER.error("Failed to start gRPC server during FlinkPollingWriter initialization.", e); + throw new RuntimeException("Could not start gRPC server", e); + } + // --- END: New logic --- + } + + /** + * [Producer side] Receives row change events from the data source, applies rate limiting, + * converts them, and places them into the in-memory queue. + * + * @param event The row change event + * @return always returns true, unless an interruption occurs. + */ + @Override + public boolean writeRow(RowChangeEvent event) { + if (event == null) { + LOGGER.warn("Received a null RowChangeEvent, skipping."); + return false; + } + + try { + // 1. Acquire a token to respect the rate limit before processing data. + // This is a blocking operation that will effectively control the upstream write speed. + rateLimiter.acquire(1); + + // 2. Convert Flink's RowChangeEvent to the gRPC RowRecord Protobuf object + SinkProto.RowRecord rowRecord = event.getRowRecord(); + + // 3. Find the corresponding queue for the table name, creating a new one atomically if it doesn't exist. + BlockingQueue queue = tableQueues.computeIfAbsent( + event.getFullTableName(), + k -> new LinkedBlockingQueue<>() // Default to an unbounded queue + ); + + // 4. Put the converted record into the queue. + queue.put(rowRecord); + + LOGGER.debug("Enqueued a row for table '{}'. Queue size is now {}.", event.getFullTableName(), queue.size()); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); // Restore the interrupted status + LOGGER.error("Thread was interrupted while writing row for table: " + event.getFullTableName(), e); + return false; + } catch (Exception e) { + LOGGER.error("Failed to process and write row for table: " + event.getFullTableName(), e); + return false; + } + + return true; + } + + /** + * [Consumer side] The gRPC service calls this method to pull data. + * Implements long-polling logic: if the queue is empty, it blocks for a specified timeout. + * batchSize acts as an upper limit on the number of records pulled to prevent oversized RPC responses. + * + * @param tableName The name of the table to pull data from + * @param batchSize The maximum number of records to pull + * @param timeout The maximum time to wait for data + * @param unit The time unit for the timeout + * @return A list of RowRecords, which will be empty if no data is available before the timeout. + * @throws InterruptedException if the thread is interrupted while waiting + */ + public List pollRecords(String tableName, int batchSize, long timeout, TimeUnit unit) + throws InterruptedException { + List records = new ArrayList<>(batchSize); + BlockingQueue queue = tableQueues.get(tableName); + + if (queue == null) { + // If the queue doesn't exist yet, wait for the specified timeout to simulate polling behavior. + unit.sleep(timeout); + return records; + } + + // Wait for the first record for up to the timeout period. + SinkProto.RowRecord firstRecord = queue.poll(timeout, unit); + if (firstRecord == null) { + // Timeout occurred, no records available. + return records; + } + + // At least one record was found, add it to the list. + records.add(firstRecord); + // Drain any remaining records up to the batch size limit without blocking. + queue.drainTo(records, batchSize - 1); + + LOGGER.info("Polled {} records for table '{}'.", records.size(), tableName); + return records; + } + + /** + * This implementation does not involve transactions, so this method is a no-op. + */ + @Override + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { + return true; + } + + /** + * This implementation uses an in-memory queue, so data is immediately available. flush is a no-op. + */ + @Override + public void flush() { + // No-op + } + + /** + * Cleans up resources on close. This is where we stop the gRPC server. + */ + @Override + public void close() throws IOException + { + LOGGER.info("Closing FlinkPollingWriter..."); + if (this.pollingRpcServer != null) { + LOGGER.info("Attempting to shut down the gRPC Polling Server..."); + this.pollingRpcServer.stop(); + LOGGER.info("gRPC Polling Server shut down."); + } + LOGGER.info("Clearing all table queues."); + tableQueues.clear(); + LOGGER.info("FlinkPollingWriter closed."); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java new file mode 100644 index 0000000..31b94bd --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java @@ -0,0 +1,93 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package io.pixelsdb.pixels.sink.writer.flink; + +import io.grpc.stub.StreamObserver; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.PixelsPollingServiceGrpc; +import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; +import io.pixelsdb.pixels.sink.util.DataTransform; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * 实现了 .proto 文件中定义的 PixelsPollingService 服务。 + * 它处理来自客户端的 PollRequest,并从 FlinkPollingWriter 中拉取数据进行响应。 + */ +// *** 核心修复: 继承自 gRPC 生成的基类 *** +public class PixelsPollingServiceImpl extends PixelsPollingServiceGrpc.PixelsPollingServiceImplBase { + private static final Logger LOGGER = LoggerFactory.getLogger(PixelsPollingServiceImpl.class); + private final FlinkPollingWriter writer; + private final int pollBatchSize; + private final long pollTimeoutMs; + /** + * 构造函数,注入 FlinkPollingWriter 并初始化服务器端配置。 + * @param writer 数据缓冲区的实例。 + */ + public PixelsPollingServiceImpl(FlinkPollingWriter writer) { + if (writer == null) { + throw new IllegalArgumentException("FlinkPollingWriter cannot be null."); + } + this.writer = writer; + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + this.pollBatchSize = config.getCommitBatchSize(); + this.pollTimeoutMs = config.getTimeoutMs(); + LOGGER.info("PixelsPollingServiceImpl initialized. Using 'sink.commit.batch.size' for pollBatchSize ({}) " + + "and 'sink.timeout.ms' for pollTimeoutMs ({}).", + this.pollBatchSize, this.pollTimeoutMs); + } + + @Override + public void pollEvents(SinkProto.PollRequest request, StreamObserver responseObserver) { + String fullTableName = request.getSchemaName() + "." + request.getTableName(); + LOGGER.debug("Received poll request for table '{}'", fullTableName); + try { + List records = writer.pollRecords( + fullTableName, + pollBatchSize, + pollTimeoutMs, + TimeUnit.MILLISECONDS + ); + List updatedRecords = DataTransform.updateRecordTimestamp( + records, + System.currentTimeMillis() + ); + SinkProto.PollResponse.Builder responseBuilder = SinkProto.PollResponse.newBuilder(); + if (updatedRecords != null && !updatedRecords.isEmpty()) { + responseBuilder.addAllRecords(updatedRecords); + } + responseObserver.onNext(responseBuilder.build()); + responseObserver.onCompleted(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.error("Polling thread was interrupted for table: " + fullTableName, e); + responseObserver.onError(io.grpc.Status.INTERNAL + .withDescription("Server polling was interrupted") + .asRuntimeException()); + } catch (Exception e) { + LOGGER.error("An unexpected error occurred while polling for table: " + fullTableName, e); + responseObserver.onError(io.grpc.Status.UNKNOWN + .withDescription("An unexpected error occurred: " + e.getMessage()) + .asRuntimeException()); + } + } +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java new file mode 100644 index 0000000..c80d421 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java @@ -0,0 +1,90 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package io.pixelsdb.pixels.sink.writer.flink; + +import io.grpc.Server; +import io.grpc.ServerBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * 负责管理 gRPC Server 的生命周期(启动、关闭)。 + * 这个类的实现与具体的 .proto 服务定义解耦。 + */ +public class PollingRpcServer { + + private static final Logger LOGGER = LoggerFactory.getLogger(PollingRpcServer.class); + private final Server server; + private final int port; + + /** + * 构造函数。 + * @param serviceImpl gRPC 服务的实现实例 + * @param port 服务器监听的端口 + */ + public PollingRpcServer(PixelsPollingServiceImpl serviceImpl, int port) { + this.port = port; + this.server = ServerBuilder.forPort(port) + .addService(serviceImpl) // 将具体的服务实现绑定到服务器 + .build(); + } + + /** + * 启动 gRPC 服务器。 + * @throws IOException 如果端口绑定失败 + */ + public void start() throws IOException { + server.start(); + LOGGER.info("gRPC Polling Server started, listening on port " + port); + } + + /** + * 平滑地关闭服务器。 + */ + public void stop() { + LOGGER.info("Attempting to shut down gRPC Polling Server..."); + if (server != null) { + try { + if (!server.isTerminated()) { + server.shutdown().awaitTermination(5, TimeUnit.SECONDS); + } + } catch (InterruptedException e) { + LOGGER.error("gRPC server shutdown interrupted.", e); + Thread.currentThread().interrupt(); + } finally { + if (!server.isTerminated()) { + LOGGER.warn("gRPC server did not terminate gracefully. Forcing shutdown."); + server.shutdownNow(); + } + } + } + LOGGER.info("gRPC Polling Server shut down."); + } + + /** + * 阻塞当前线程,直到 gRPC 服务器关闭。 + * 通常在主线程中调用,以防止应用退出。 + */ + public void awaitTermination() throws InterruptedException { + if (server != null) { + server.awaitTermination(); + } + } +} diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/RpcEndToEndTest.java b/src/test/java/io/pixelsdb/pixels/sink/writer/RpcEndToEndTest.java new file mode 100644 index 0000000..0a86e12 --- /dev/null +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/RpcEndToEndTest.java @@ -0,0 +1,224 @@ +/* + * Copyright 2025 PixelsDB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package io.pixelsdb.pixels.sink.writer; + +import com.google.protobuf.ByteString; +import io.grpc.ManagedChannel; +import io.grpc.ManagedChannelBuilder; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; +import io.pixelsdb.pixels.sink.PixelsPollingServiceGrpc; +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.CommandLineConfig; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.source.SinkSource; +import io.pixelsdb.pixels.sink.source.SinkSourceFactory; +import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.writer.retina.SinkContextManager; +import io.pixelsdb.pixels.sink.writer.retina.TransactionProxy; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriterFactory; +import io.pixelsdb.pixels.sink.util.MetricsFacade; +import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.hotspot.DefaultExports; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +public class RpcEndToEndTest +{ + private static final int TEST_PORT = 9091; + private static final int RECORD_COUNT = 5; + private static SinkSource sinkSource; // Keep a reference to stop it later + private static final String TEST_SCHEMA = "pixels_bench_sf10x"; + private static final String TEST_TABLE = "savingaccount"; + private static final String FULL_TABLE_NAME = TEST_SCHEMA + "." + TEST_TABLE; + private static final String CONFIG_FILE_PATH = "conf/pixels-sink.aws.properties"; + private static HTTPServer prometheusHttpServer; + // This method contains the setup logic from PixelsSinkApp + private static void startServer() + { + System.out.println("[SETUP] Starting server with full initialization sequence..."); + try { + // === 1. Mimic init() method from PixelsSinkApp === + System.out.println("[SETUP] Initializing configuration from " + CONFIG_FILE_PATH + "..."); + PixelsSinkConfigFactory.initialize(CONFIG_FILE_PATH); + + System.out.println("[SETUP] Initializing MetricsFacade..."); + MetricsFacade.getInstance().setSinkContextManager(SinkContextManager.getInstance()); + + // For determinism in testing, override the port from the config file + System.setProperty("sink.flink.server.port", String.valueOf(TEST_PORT)); + // === 2. Mimic main() method from PixelsSinkApp === + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + System.out.println("[SETUP] Creating SinkSource application engine..."); + sinkSource = SinkSourceFactory.createSinkSource(); + System.out.println("[SETUP] Setting up Prometheus monitoring server..."); + if (config.isMonitorEnabled()) { + DefaultExports.initialize(); + // To avoid port conflicts during tests, you could use port 0 for a random port + // For this example, we'll assume the config port is fine. + prometheusHttpServer = new HTTPServer(config.getMonitorPort()); + System.out.println("[SETUP] Prometheus server started on port: " + config.getMonitorPort()); + } + // === 3. THE MOST CRITICAL STEP: Start the processor === + System.out.println("[SETUP] Starting SinkSource processor threads..."); + sinkSource.start(); + System.out.println("[SETUP] Server is fully initialized and running."); + } catch (IOException e) { + // If setup fails, we cannot run the test. + throw new RuntimeException("Failed to start the server for the test", e); + } + } + public static void main(String[] args) throws InterruptedException, IOException + { + // ========== 1. SETUP PHASE ========== + // Start the server components within this same process. + startServer(); + + // Give the server a moment to fully start up and bind to the port. + Thread.sleep(2000); // 2 seconds, adjust if needed + // [REFACTORED] To ensure the FlinkPollingWriter uses our test port, + // we set it as a system property before the writer is created. + // The PixelsSinkConfigFactory should be configured to read this property. + System.setProperty("sink.flink.server.port", String.valueOf(TEST_PORT)); + + // [REFACTORED] The setup is now dramatically simpler. + // Instantiating FlinkPollingWriter is the ONLY step needed. + // Its constructor automatically creates the service and starts the gRPC server. + System.out.println("[SETUP] Initializing FlinkPollingWriter, which will start the gRPC server..."); + PixelsSinkWriter writer = PixelsSinkWriterFactory.getWriter(); + + // [REFACTORED] The following lines are no longer needed and have been removed: + // PixelsPollingServiceImpl serviceImpl = new PixelsPollingServiceImpl(writer); + // PollingRpcServer server = new PollingRpcServer(serviceImpl, TEST_PORT); + // server.start(); + + ExecutorService executor = Executors.newFixedThreadPool(2); + + try { + // 3. Start the mock data producer (in a separate thread) + executor.submit(() -> { + try { + System.out.println("[PRODUCER] Starting mock data producer..."); + // Simulate producing 5 INSERT records + for (int i = 1; i <= 5; i++) { + Thread.sleep(1000); // Simulate data production interval + + SinkProto.SourceInfo sourceInfo = SinkProto.SourceInfo.newBuilder() + .setDb(TEST_SCHEMA) + .setTable(TEST_TABLE) + .build(); + + byte[] idBytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8); + SinkProto.ColumnValue idColumnValue = SinkProto.ColumnValue.newBuilder() + .setValue(ByteString.copyFrom(idBytes)) + .build(); + + SinkProto.RowValue afterImage = SinkProto.RowValue.newBuilder() + .addValues(idColumnValue) + .build(); + + SinkProto.RowRecord record = SinkProto.RowRecord.newBuilder() + .setSource(sourceInfo) + .setOp(SinkProto.OperationType.INSERT) + .setAfter(afterImage) + .build(); + + RowChangeEvent event = new RowChangeEvent(record); + System.out.printf("[PRODUCER] Writing INSERT record %d for table '%s.%s'%n", i, TEST_SCHEMA, TEST_TABLE); + writer.writeRow(event); + } + System.out.println("[PRODUCER] Finished writing data."); + } catch (InterruptedException e) { + System.err.println("[PRODUCER] Producer thread was interrupted."); + Thread.currentThread().interrupt(); + } catch (SinkException e) { + System.err.println("[PRODUCER] SinkException occurred: %s" + e.getMessage()); + } + }); + + // 4. Start the gRPC client (in another thread) + executor.submit(() -> { + System.out.println("[CLIENT] Starting gRPC client..."); + ManagedChannel channel = ManagedChannelBuilder.forAddress("localhost", TEST_PORT) + .usePlaintext() + .build(); + + try { + PixelsPollingServiceGrpc.PixelsPollingServiceBlockingStub client = PixelsPollingServiceGrpc.newBlockingStub(channel); + int recordsPolled = 0; + int maxRetries = 10; + int retryCount = 0; + while (recordsPolled < 5 && retryCount < maxRetries) { + System.out.println("[CLIENT] Sending poll request for table '" + FULL_TABLE_NAME + "'..."); + SinkProto.PollRequest request = SinkProto.PollRequest.newBuilder() + .setSchemaName(TEST_SCHEMA) + .setTableName(TEST_TABLE) + .build(); + SinkProto.PollResponse response = client.pollEvents(request); + + if (response.getRecordsCount() > 0) { + System.out.printf("[CLIENT] SUCCESS: Polled %d record(s).%n", response.getRecordsCount()); + response.getRecordsList().forEach(record -> System.out.println(" -> " + record.toString().trim())); + recordsPolled += response.getRecordsCount(); + } else { + System.out.println("[CLIENT] Polled 0 records, will retry..."); + retryCount++; + } + } + if (recordsPolled == 5) { + System.out.println("[CLIENT] Test finished successfully. Polled all 5 records."); + } else { + System.err.println("[CLIENT] Test FAILED. Did not poll all records in time."); + } + } finally { + channel.shutdownNow(); + } + }); + } finally { + // 5. Wait for the test to complete and clean up resources + executor.shutdown(); + if (!executor.awaitTermination(1, TimeUnit.MINUTES)) { + System.err.println("[CLEANUP] Test timed out."); + executor.shutdownNow(); + } else { + System.out.println("[CLEANUP] Test completed."); + } + + // [REFACTORED] The correct way to stop the server is now by closing the writer. + writer.close(); + System.out.println("[CLEANUP] FlinkPollingWriter closed and server stopped."); + } + // ========== 3. CLEANUP PHASE ========== + System.out.println("[CLEANUP] Tearing down server components..."); + if (sinkSource != null) { + sinkSource.stopProcessor(); + System.out.println("[CLEANUP] SinkSource processor stopped."); + // Note: The writer is part of sinkSource, and its resources + // should be cleaned up by stopProcessor or an equivalent close method. + } + if (prometheusHttpServer != null) { + prometheusHttpServer.close(); + System.out.println("[CLEANUP] Prometheus server stopped."); + } + System.out.println("[CLEANUP] Test finished."); + } +} From a1aefb1e1e359c57ff4bb9b1a3097dbf20bee4aa Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 4 Dec 2025 09:32:27 +0000 Subject: [PATCH 39/53] Change License to AGPL --- conf/pixels-sink.aws.properties | 12 ++++-- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 22 +++++----- .../pixels/sink/PixelsSinkProvider.java | 23 ++++++----- .../pixels/sink/config/CommandLineConfig.java | 23 ++++++----- .../pixels/sink/config/ConfigKey.java | 24 ++++++----- .../pixels/sink/config/ConfigLoader.java | 24 ++++++----- .../pixels/sink/config/PixelsSinkConfig.java | 22 +++++----- .../sink/config/PixelsSinkConstants.java | 23 ++++++----- .../sink/config/PixelsSinkDefaultConfig.java | 23 ++++++----- .../pixels/sink/config/TransactionConfig.java | 23 ++++++----- .../sink/config/factory/KafkaPropFactory.java | 23 ++++++----- .../factory/KafkaPropFactorySelector.java | 23 ++++++----- .../factory/PixelsSinkConfigFactory.java | 23 ++++++----- .../factory/RowRecordKafkaPropFactory.java | 23 ++++++----- .../factory/TransactionKafkaPropFactory.java | 23 ++++++----- .../pixels/sink/event/RowChangeEvent.java | 23 ++++++----- .../event/deserializer/DeserializerUtil.java | 23 ++++++----- .../RowChangeEventAvroDeserializer.java | 23 ++++++----- .../RowChangeEventJsonDeserializer.java | 23 ++++++----- .../RowChangeEventStructDeserializer.java | 24 ++++++----- .../event/deserializer/RowDataParser.java | 23 ++++++----- .../deserializer/SchemaDeserializer.java | 23 ++++++----- .../TransactionAvroMessageDeserializer.java | 23 ++++++----- .../TransactionJsonMessageDeserializer.java | 23 ++++++----- .../TransactionStructMessageDeserializer.java | 24 ++++++----- .../pixels/sink/exception/SinkException.java | 1 + .../sink/freshness/FreshnessClient.java | 2 +- .../pixels/sink/metadata/TableMetadata.java | 23 ++++++----- .../sink/metadata/TableMetadataRegistry.java | 23 ++++++----- .../sink/processor/MonitorThreadManager.java | 23 ++++++----- .../sink/processor/StoppableProcessor.java | 22 +++++----- .../pixels/sink/processor/TableProcessor.java | 24 ++++++----- .../pixels/sink/processor/TopicProcessor.java | 23 ++++++----- .../sink/processor/TransactionProcessor.java | 24 ++++++----- .../pixels/sink/provider/EventProvider.java | 23 ++++++----- .../pixels/sink/provider/ProtoType.java | 24 ++++++----- .../provider/TableEventEngineProvider.java | 24 ++++++----- .../provider/TableEventKafkaProvider.java | 24 ++++++----- .../sink/provider/TableEventProvider.java | 23 ++++++----- .../TableEventStorageLoopProvider.java | 20 ++++++++++ .../provider/TableEventStorageProvider.java | 24 ++++++----- ...leProviderAndProcessorPipelineManager.java | 24 ++++++----- .../TransactionEventEngineProvider.java | 24 ++++++----- .../TransactionEventKafkaProvider.java | 24 ++++++----- .../provider/TransactionEventProvider.java | 23 ++++++----- .../TransactionEventStorageLoopProvider.java | 23 ++++++----- .../TransactionEventStorageProvider.java | 23 ++++++----- .../source/AbstractSinkStorageSource.java | 20 ++++++++++ .../sink/source/FasterSinkStorageSource.java | 24 ++++++----- .../sink/source/LegacySinkStorageSource.java | 24 ++++++----- .../sink/source/PixelsDebeziumConsumer.java | 24 ++++++----- .../pixels/sink/source/SinkEngineSource.java | 26 +++++++----- .../pixels/sink/source/SinkKafkaSource.java | 23 ++++++----- .../pixels/sink/source/SinkSource.java | 22 +++++----- .../pixels/sink/source/SinkSourceFactory.java | 22 +++++----- .../pixels/sink/util/BlockingBoundedMap.java | 23 ++++++----- .../pixels/sink/util/DataTransform.java | 40 +++++++++---------- .../pixelsdb/pixels/sink/util/DateUtil.java | 24 ++++++----- .../pixels/sink/util/EtcdFileRegistry.java | 24 ++++++----- .../pixels/sink/util/FlushRateLimiter.java | 23 ++++++----- .../pixels/sink/util/LatencySimulator.java | 23 ++++++----- .../pixels/sink/util/MetricsFacade.java | 23 ++++++----- .../pixels/sink/util/OneSecondAverage.java | 24 ++++++----- .../pixels/sink/util/TableCounters.java | 20 ++++++++++ .../pixels/sink/writer/CsvWriter.java | 22 +++++----- .../pixels/sink/writer/NoneWriter.java | 23 ++++++----- .../pixels/sink/writer/PixelsSinkMode.java | 23 ++++++----- .../pixels/sink/writer/PixelsSinkWriter.java | 23 ++++++----- .../sink/writer/PixelsSinkWriterFactory.java | 23 ++++++----- .../pixels/sink/writer/ProtoWriter.java | 24 ++++++----- .../sink/writer/RotatingWriterManager.java | 24 ++++++----- .../sink/writer/flink/FlinkPollingWriter.java | 22 +++++----- .../flink/PixelsPollingServiceImpl.java | 22 +++++----- .../sink/writer/flink/PollingRpcServer.java | 22 +++++----- .../writer/retina/RetinaServiceProxy.java | 23 ++++++----- .../sink/writer/retina/RetinaWriter.java | 23 ++++++----- .../sink/writer/retina/SinkContext.java | 23 ++++++----- .../writer/retina/SinkContextManager.java | 23 ++++++----- .../writer/retina/TableCrossTxWriter.java | 24 ++++++----- .../retina/TableSingleRecordWriter.java | 20 ++++++++++ .../writer/retina/TableSingleTxWriter.java | 24 ++++++----- .../sink/writer/retina/TableWriter.java | 24 ++++++----- .../sink/writer/retina/TableWriterProxy.java | 23 ++++++----- .../sink/writer/retina/TransactionMode.java | 23 ++++++----- .../sink/writer/retina/TransactionProxy.java | 23 ++++++----- 85 files changed, 1115 insertions(+), 809 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index c96476d..0042e90 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -3,12 +3,16 @@ sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet sink.datasource.rate.limit=150000 # Sink Config: retina | csv | proto | flink | none -sink.mode=FLINK +sink.mode=flink sink.retina.client=2 sink.monitor.report.enable=false -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/rate150k_6.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness150k_6.csv -# row or txn +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_test.csv +# trino for freshness query +trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x +trino.user=pixels +trino.password=password +# row or txn or embed sink.monitor.freshness.level=row sink.storage.loop=true # Kafka Config diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 9689bda..2480983 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink; import io.pixelsdb.pixels.sink.config.CommandLineConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java index 057a7d4..a72212d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink; import io.pixelsdb.pixels.common.sink.SinkProvider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java index ce8c6e8..d2d2481 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java index 066c028..917cad3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.config; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java index 5f9d760..b3795cb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.config; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 0eecdc4..c297707 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.config; import io.pixelsdb.pixels.common.utils.ConfigFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java index 0f3ed12..481e5ca 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config; public final class PixelsSinkConstants diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java index 0ea3482..57dec10 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config; public class PixelsSinkDefaultConfig diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java index 7bf851c..cfe1278 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config; import io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java index f3fabfb..183851d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java index 54f5d69..50fb4c4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java index 03c69de..268ae4b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.common.utils.ConfigFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java index cb052e2..e6bb211 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java index 1c678a3..7b50c46 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 0f92800..0298e1e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event; import com.google.common.hash.Hashing; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java index 50bb42c..d47242a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java index 0f97664..a451235 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import io.apicurio.registry.serde.SerdeConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java index 94b126f..3a850d1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java index 9129885..5154bd1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.event.deserializer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java index 7d719a9..bce8871 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java index d5d8b79..02be8ca 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java index 16f1852..61b1440 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import io.apicurio.registry.serde.SerdeConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java index d7eca6b..924b43c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java index 7ad0c1c..9af8c37 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.event.deserializer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java b/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java index 1160cf5..cb19398 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java +++ b/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java @@ -17,6 +17,7 @@ * License along with Pixels. If not, see * . */ + package io.pixelsdb.pixels.sink.exception; public class SinkException extends Exception diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 2bddb04..e7aed27 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.freshness; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java index 24a769d..25d649d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.metadata; import io.pixelsdb.pixels.common.exception.MetadataException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index daac3ef..4d1212e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.metadata; import io.pixelsdb.pixels.common.exception.MetadataException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java index 4ee0052..b5f78c9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java index 61583d8..b35e022 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.processor; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 9a91030..4317d0f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.processor; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java index 26138a9..5069236 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java index 3a50a46..1841168 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, - * either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index 97315db..1720a0e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.util.MetricsFacade; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java index e7f50be..5841614 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java index dccaaf5..494d518 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java index 6aa68fb..12ec3e4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, - * either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java index fc4e48a..73f6d66 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java index d30bb36..7b98718 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.provider; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java index f4d0af1..6703625 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index f8afd6f..6e8d9c3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java index f135342..57feefc 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java index 95681db..bb5f49f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.provider; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java index 8072d53..413eb64 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java index afa0b38..d225ea0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.provider; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java index 4ecd555..3527268 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.provider; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index fb04de9..c3ebaa5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.source; import io.pixelsdb.pixels.common.physical.PhysicalReader; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java index 2dcbfc3..6026c5f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/FasterSinkStorageSource.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.source; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java index c62887a..baa2153 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.source; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java index e5a9c29..f936f7f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.source; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java index 1aa3bec..8c9d548 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java @@ -1,20 +1,24 @@ -package io.pixelsdb.pixels.sink.source;/* +/* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + +package io.pixelsdb.pixels.sink.source; import io.debezium.embedded.Connect; import io.debezium.engine.DebeziumEngine; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java index 4f54742..c2e696b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.source; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java index 6d8e010..274fd93 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.source; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java index fde204d..dece1f8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.source; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java index 67bde68..dc41b50 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.util; import java.util.concurrent.ConcurrentHashMap; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java index d1daa8d..595675e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -1,23 +1,23 @@ - /* - * Copyright 2025 PixelsDB. - * - * This file is part of Pixels. - * - * Pixels is free software: you can redistribute it and/or modify - * it under the terms of the Affero GNU General Public License as - * published by the Free Software Foundation, either version 3 of - * the License, or (at your option) any later version. - * - * Pixels is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * Affero GNU General Public License for more details. - * - * You should have received a copy of the Affero GNU General Public - * License along with Pixels. If not, see - * . - */ - +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.util; import com.google.protobuf.ByteString; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java b/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java index 115d585..2fadbcb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.util; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java index 8f3206c..fbafa43 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.util; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java index 4443c80..70608e4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.util; import com.google.common.util.concurrent.RateLimiter; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java b/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java index 90e598c..7bb9ce8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.util; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 4c15d2c..bcb5e9d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.util; import com.google.protobuf.ByteString; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java b/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java index c430838..555ddf5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.util; import java.util.ArrayDeque; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java b/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java index b8f1c2c..5652c64 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.util; /** diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java index 1a16f2c..b577d2f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.writer; import com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index 5268bc5..41c73c0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java index af9962c..882e26f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java index 744b46e..72ca26b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java index 2443444..7e39555 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java index fcd2c42..6fd534f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.writer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java index a873892..4dc802a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.writer; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java index 07d3ef9..f3e566c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.writer.flink; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java index 31b94bd..eee18c4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.writer.flink; import io.grpc.stub.StreamObserver; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java index c80d421..cba2382 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PollingRpcServer.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ + package io.pixelsdb.pixels.sink.writer.flink; import io.grpc.Server; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java index 216458c..ef00e9c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.RetinaException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 2de7423..96983a5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.sink.SinkProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 81595dd..ef19264 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.transaction.TransContext; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index bac9f03..c556a46 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.TransException; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 6019223..0894c62 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.writer.retina; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java index 3bf4702..9671c16 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.transaction.TransContext; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java index 242da74..b757845 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java @@ -1,19 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.retina.RetinaProto; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index fa43cbf..c1f0fae 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -1,21 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - - + package io.pixelsdb.pixels.sink.writer.retina; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java index 88353f6..0fe7f17 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.node.BucketCache; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java index e424332..8a873a3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; public enum TransactionMode diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index e63b602..db55f88 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -1,20 +1,23 @@ /* * Copyright 2025 PixelsDB. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * This file is part of Pixels. * - * http://www.apache.org/licenses/LICENSE-2.0 + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.TransException; From 9e8b46242d85b282ca6913913ff7fa82c24c3c24 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Sun, 7 Dec 2025 08:55:36 +0000 Subject: [PATCH 40/53] small fix --- .../sink/config/PixelsSinkConstants.java | 2 +- .../pixels/sink/event/RowChangeEvent.java | 19 +++++++--- .../pixels/sink/provider/EventProvider.java | 6 +-- .../source/AbstractSinkStorageSource.java | 4 +- .../sink/writer/flink/FlinkPollingWriter.java | 20 +++------- .../flink/PixelsPollingServiceImpl.java | 37 +++++++++++++------ 6 files changed, 51 insertions(+), 37 deletions(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java index 481e5ca..adeb4eb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java @@ -25,7 +25,7 @@ public final class PixelsSinkConstants public static final String ROW_RECORD_KAFKA_PROP_FACTORY = "row-record"; public static final String TRANSACTION_KAFKA_PROP_FACTORY = "transaction"; public static final int MONITOR_NUM = 2; - + public static final int MAX_QUEUE_SIZE = 10_000; public static final String SNAPSHOT_TX_PREFIX = "SNAPSHOT-"; private PixelsSinkConstants() diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index 0298e1e..a7afa2a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -22,6 +22,7 @@ import com.google.common.hash.Hashing; import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.common.node.BucketCache; import io.pixelsdb.pixels.common.utils.RetinaUtils; @@ -75,15 +76,17 @@ public class RowChangeEvent private boolean indexKeyInited = false; @Getter - private final long tableId; + private long tableId; + + @Getter + private SchemaTableName schemaTableName; public RowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException { this.rowRecord = rowRecord; TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); this.schema = tableMetadataRegistry.getTypeDescription(getSchemaName(), getTable()); - this.tableId = tableMetadataRegistry.getTableId(getSchemaName(), getTable()); - initColumnValueMap(); + init(); initIndexKey(); } @@ -92,10 +95,17 @@ public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) thr this.rowRecord = rowRecord; this.schema = schema; + init(); + // initIndexKey(); + } + + private void init() throws SinkException + { TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); this.tableId = tableMetadataRegistry.getTableId(getSchemaName(), getTable()); + this.schemaTableName = new SchemaTableName(getSchemaName(), getTable()); + initColumnValueMap(); - // initIndexKey(); } private void initColumnValueMap() @@ -258,7 +268,6 @@ public String getFullTableName() return rowRecord.getSource().getSchema() + "." + rowRecord.getSource().getTable(); // return getSchemaName() + "." + getTable(); } - // TODO(AntiO2): How to Map Schema Names Between Source DB and Pixels public String getSchemaName() { diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index 1720a0e..b520c37 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -20,6 +20,7 @@ package io.pixelsdb.pixels.sink.provider; +import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,10 +37,9 @@ public abstract class EventProvider implements private static final int BATCH_SIZE = 64; private static final int THREAD_NUM = 4; private static final long MAX_WAIT_MS = 5; // configurable - private static final int MAX_QUEUE_SIZE = 10_000; protected final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(MAX_QUEUE_SIZE); - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(MAX_QUEUE_SIZE); + private final BlockingQueue rawEventQueue = new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE); + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE); private final ExecutorService decodeExecutor = Executors.newFixedThreadPool(THREAD_NUM); private Thread providerThread; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index c3ebaa5..b0505ce 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -25,6 +25,7 @@ import io.pixelsdb.pixels.common.physical.Storage; import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; @@ -63,7 +64,6 @@ public abstract class AbstractSinkStorageSource implements SinkSource private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final TableProviderAndProcessorPipelineManager> tablePipelineManager = new TableProviderAndProcessorPipelineManager<>(); private final boolean storageLoopEnabled; - private final int MAX_QUEUE_SIZE = 10_000; private final FlushRateLimiter sourceRateLimiter; protected TransactionEventStorageLoopProvider> transactionEventProvider; protected TransactionProcessor transactionProcessor; @@ -151,7 +151,7 @@ public void start() // Get or create queue BlockingQueue, Integer>> queue = queueMap.computeIfAbsent(key, - k -> new LinkedBlockingQueue<>(MAX_QUEUE_SIZE)); + k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE)); // Put future in queue queue.put(new Pair<>(valueFuture, loopId)); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java index f3e566c..10e82e5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java @@ -20,11 +20,12 @@ package io.pixelsdb.pixels.sink.writer.flink; +import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; +import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,10 +51,7 @@ public class FlinkPollingWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(FlinkPollingWriter.class); // Core data structure: A thread-safe map from table name to a thread-safe blocking queue. - private final Map> tableQueues; - - // Ingress rate limiter to control the data writing speed. - private final FlushRateLimiter rateLimiter; + private final Map> tableQueues; // The gRPC server instance managed by this writer. private final PollingRpcServer pollingRpcServer; @@ -64,8 +62,6 @@ public class FlinkPollingWriter implements PixelsSinkWriter { */ public FlinkPollingWriter() { this.tableQueues = new ConcurrentHashMap<>(); - // Get the global RateLimiter instance - this.rateLimiter = FlushRateLimiter.getInstance(); LOGGER.info("FlinkPollingWriter initialized with FlushRateLimiter."); // --- START: New logic to initialize and start the gRPC server --- @@ -106,17 +102,13 @@ public boolean writeRow(RowChangeEvent event) { } try { - // 1. Acquire a token to respect the rate limit before processing data. - // This is a blocking operation that will effectively control the upstream write speed. - rateLimiter.acquire(1); - // 2. Convert Flink's RowChangeEvent to the gRPC RowRecord Protobuf object SinkProto.RowRecord rowRecord = event.getRowRecord(); // 3. Find the corresponding queue for the table name, creating a new one atomically if it doesn't exist. BlockingQueue queue = tableQueues.computeIfAbsent( - event.getFullTableName(), - k -> new LinkedBlockingQueue<>() // Default to an unbounded queue + event.getSchemaTableName(), + k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE) // Default to an unbounded queue ); // 4. Put the converted record into the queue. @@ -148,7 +140,7 @@ public boolean writeRow(RowChangeEvent event) { * @return A list of RowRecords, which will be empty if no data is available before the timeout. * @throws InterruptedException if the thread is interrupted while waiting */ - public List pollRecords(String tableName, int batchSize, long timeout, TimeUnit unit) + public List pollRecords(SchemaTableName tableName, int batchSize, long timeout, TimeUnit unit) throws InterruptedException { List records = new ArrayList<>(batchSize); BlockingQueue queue = tableQueues.get(tableName); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java index eee18c4..49e15fc 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java @@ -21,15 +21,19 @@ package io.pixelsdb.pixels.sink.writer.flink; import io.grpc.stub.StreamObserver; +import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.PixelsPollingServiceGrpc; +import io.pixelsdb.pixels.sink.PixelsPollingServiceGrpc; +import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; import io.pixelsdb.pixels.sink.util.DataTransform; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.xml.validation.Schema; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; @@ -43,6 +47,7 @@ public class PixelsPollingServiceImpl extends PixelsPollingServiceGrpc.PixelsPol private final FlinkPollingWriter writer; private final int pollBatchSize; private final long pollTimeoutMs; + private final FlushRateLimiter flushRateLimiter; /** * 构造函数,注入 FlinkPollingWriter 并初始化服务器端配置。 * @param writer 数据缓冲区的实例。 @@ -55,6 +60,7 @@ public PixelsPollingServiceImpl(FlinkPollingWriter writer) { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.pollBatchSize = config.getCommitBatchSize(); this.pollTimeoutMs = config.getTimeoutMs(); + this.flushRateLimiter = FlushRateLimiter.getInstance(); LOGGER.info("PixelsPollingServiceImpl initialized. Using 'sink.commit.batch.size' for pollBatchSize ({}) " + "and 'sink.timeout.ms' for pollTimeoutMs ({}).", this.pollBatchSize, this.pollTimeoutMs); @@ -62,33 +68,40 @@ public PixelsPollingServiceImpl(FlinkPollingWriter writer) { @Override public void pollEvents(SinkProto.PollRequest request, StreamObserver responseObserver) { - String fullTableName = request.getSchemaName() + "." + request.getTableName(); - LOGGER.debug("Received poll request for table '{}'", fullTableName); + SchemaTableName schemaTableName = new SchemaTableName(request.getSchemaName(), request.getTableName()); + LOGGER.debug("Received poll request for table '{}'", schemaTableName); try { List records = writer.pollRecords( - fullTableName, + schemaTableName, pollBatchSize, pollTimeoutMs, TimeUnit.MILLISECONDS ); - List updatedRecords = DataTransform.updateRecordTimestamp( - records, - System.currentTimeMillis() - ); + SinkProto.PollResponse.Builder responseBuilder = SinkProto.PollResponse.newBuilder(); - if (updatedRecords != null && !updatedRecords.isEmpty()) { - responseBuilder.addAllRecords(updatedRecords); + if(records != null) + { + List updatedRecords = updatedRecords = DataTransform.updateRecordTimestamp( + records, + System.currentTimeMillis() + ); + + if (updatedRecords != null && !updatedRecords.isEmpty()) { + responseBuilder.addAllRecords(updatedRecords); + this.flushRateLimiter.acquire(updatedRecords.size()); + } } + responseObserver.onNext(responseBuilder.build()); responseObserver.onCompleted(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOGGER.error("Polling thread was interrupted for table: " + fullTableName, e); + LOGGER.error("Polling thread was interrupted for table: " + schemaTableName, e); responseObserver.onError(io.grpc.Status.INTERNAL .withDescription("Server polling was interrupted") .asRuntimeException()); } catch (Exception e) { - LOGGER.error("An unexpected error occurred while polling for table: " + fullTableName, e); + LOGGER.error("An unexpected error occurred while polling for table: " + schemaTableName, e); responseObserver.onError(io.grpc.Status.UNKNOWN .withDescription("An unexpected error occurred: " + e.getMessage()) .asRuntimeException()); From 6329209067c28ed747821c891c95aa8e5f0bf4f0 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Mon, 8 Dec 2025 07:40:25 +0000 Subject: [PATCH 41/53] IDEA FILE --- .gitignore | 1 - .idea/.gitignore | 10 ++++++++++ .idea/compiler.xml | 26 ++++++++++++++++++++++++++ .idea/encodings.xml | 7 +++++++ .idea/jarRepositories.xml | 20 ++++++++++++++++++++ .idea/misc.xml | 12 ++++++++++++ .idea/vcs.xml | 6 ++++++ 7 files changed, 81 insertions(+), 1 deletion(-) create mode 100644 .idea/.gitignore create mode 100644 .idea/compiler.xml create mode 100644 .idea/encodings.xml create mode 100644 .idea/jarRepositories.xml create mode 100644 .idea/misc.xml create mode 100644 .idea/vcs.xml diff --git a/.gitignore b/.gitignore index 15a9737..db6976d 100644 --- a/.gitignore +++ b/.gitignore @@ -22,7 +22,6 @@ # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml hs_err_pid* -.idea build target .classpath* diff --git a/.idea/.gitignore b/.idea/.gitignore new file mode 100644 index 0000000..7d05e99 --- /dev/null +++ b/.idea/.gitignore @@ -0,0 +1,10 @@ +# 默认忽略的文件 +/shelf/ +/workspace.xml +# 基于编辑器的 HTTP 客户端请求 +/httpRequests/ +# 依赖于环境的 Maven 主目录路径 +/mavenHomeManager.xml +# Datasource local storage ignored files +/dataSources/ +/dataSources.local.xml diff --git a/.idea/compiler.xml b/.idea/compiler.xml new file mode 100644 index 0000000..32f4678 --- /dev/null +++ b/.idea/compiler.xml @@ -0,0 +1,26 @@ + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/encodings.xml b/.idea/encodings.xml new file mode 100644 index 0000000..aa00ffa --- /dev/null +++ b/.idea/encodings.xml @@ -0,0 +1,7 @@ + + + + + + + \ No newline at end of file diff --git a/.idea/jarRepositories.xml b/.idea/jarRepositories.xml new file mode 100644 index 0000000..c8eb127 --- /dev/null +++ b/.idea/jarRepositories.xml @@ -0,0 +1,20 @@ + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/misc.xml b/.idea/misc.xml new file mode 100644 index 0000000..6732724 --- /dev/null +++ b/.idea/misc.xml @@ -0,0 +1,12 @@ + + + + + + + + \ No newline at end of file diff --git a/.idea/vcs.xml b/.idea/vcs.xml new file mode 100644 index 0000000..35eb1dd --- /dev/null +++ b/.idea/vcs.xml @@ -0,0 +1,6 @@ + + + + + + \ No newline at end of file From 508e5aab80eb1ee6f3397c901b3413c5a6ed56bc Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 9 Dec 2025 08:09:33 +0000 Subject: [PATCH 42/53] Trans Issue --- .idea/icon.svg | 1 + conf/pixels-sink.aws.properties | 31 +++-- .../pixels/sink/config/PixelsSinkConfig.java | 3 +- .../sink/freshness/FreshnessClient.java | 7 +- .../pixels/sink/processor/TableProcessor.java | 6 - .../TableEventStorageLoopProvider.java | 23 +++- .../source/AbstractSinkStorageSource.java | 6 +- .../pixels/sink/util/BlockingBoundedMap.java | 6 + .../pixels/sink/util/DataTransform.java | 129 ++++++++++++------ .../pixels/sink/util/MetricsFacade.java | 5 +- .../flink/PixelsPollingServiceImpl.java | 13 +- .../sink/writer/retina/SinkContext.java | 21 ++- .../writer/retina/SinkContextManager.java | 42 +++++- .../writer/retina/TableCrossTxWriter.java | 17 +-- .../retina/TableSingleRecordWriter.java | 15 +- .../sink/writer/retina/TableWriter.java | 42 ++++++ .../sink/writer/retina/TransactionProxy.java | 11 +- 17 files changed, 267 insertions(+), 111 deletions(-) create mode 100644 .idea/icon.svg diff --git a/.idea/icon.svg b/.idea/icon.svg new file mode 100644 index 0000000..ad8442b --- /dev/null +++ b/.idea/icon.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 0042e90..c236daf 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,19 +1,23 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=150000 +sink.datasource.rate.limit=10000 # Sink Config: retina | csv | proto | flink | none -sink.mode=flink -sink.retina.client=2 -sink.monitor.report.enable=false -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_test.csv +sink.mode=retina +sink.retina.client=8 +## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature +sink.trans.mode=batch +sink.monitor.report.enable=true +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/i7i_10k_dec.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_i7i_10k_dec.csv # trino for freshness query trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x trino.user=pixels trino.password=password +trino.parallel=5 # row or txn or embed -sink.monitor.freshness.level=row +sink.monitor.freshness.level=embed +sink.monitor.freshness.embed.warmup=10 sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -36,15 +40,15 @@ sink.retina.mode=stream sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 -sink.flush.interval.ms=20000 -sink.flush.batch.size=100 +sink.flush.interval.ms=200 +sink.flush.batch.size=50 sink.max.retries=3 ## writer commit # sync or async sink.commit.method=sync -sink.commit.batch.size=200 +sink.commit.batch.size=10 sink.commit.batch.worker=32 -sink.commit.batch.delay=1000 +sink.commit.batch.delay=3000 ## Proto Config sink.proto.dir=file:///home/ubuntu/disk1/hybench/ sink.proto.data=hybench10_10 @@ -58,8 +62,7 @@ transaction.topic.suffix=transaction #transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer sink.trans.batch.size=100 -## batch or single or record -sink.trans.mode=record + # Sink Metrics sink.monitor.enable=true sink.monitor.port=9465 @@ -69,7 +72,7 @@ sink.monitor.freshness.interval=1000 # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 -# debezium +# debezium engine config debezium.name=testEngine debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector debezium.provide.transaction.metadata=true diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index c297707..ff4fd29 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -179,7 +179,8 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.freshness.level", defaultValue = "row") // row or txn or embed private String sinkMonitorFreshnessLevel; - + @ConfigKey(value = "sink.monitor.freshness.embed.warmup", defaultValue = "10") + private Integer sinkMonitorFreshnessEmbedWarmupSeconds; @ConfigKey(value = "trino.url") private String trinoUrl; diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index e7aed27..ea22b38 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -49,12 +49,11 @@ public class FreshnessClient { // Key modification: Use a thread-safe Set to maintain the list of tables to monitor dynamically. private final Set monitoredTables; private static final int QUERY_INTERVAL_SECONDS = 1; - private Connection connection; private final ScheduledExecutorService scheduler; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private static volatile FreshnessClient instance; - + private final int warmUpSeconds; private FreshnessClient() { // Initializes the set with thread safety wrapper this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); @@ -63,7 +62,7 @@ private FreshnessClient() { this.trinoUser = config.getTrinoUser(); this.trinoJdbcUrl = config.getTrinoUrl(); this.trinoPassword = config.getTrinoPassword(); - + this.warmUpSeconds = config.getSinkMonitorFreshnessEmbedWarmupSeconds(); // Initializes a single-threaded scheduler for executing freshness queries this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { Thread t = Executors.defaultThreadFactory().newThread(r); @@ -176,7 +175,7 @@ public void start() { LOGGER.info("Starting Freshness Client, querying every {} seconds.", QUERY_INTERVAL_SECONDS); scheduler.scheduleAtFixedRate(this::queryAndCalculateFreshness, - 0, // Initial delay + warmUpSeconds, QUERY_INTERVAL_SECONDS, TimeUnit.SECONDS); } catch (SQLException e) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 4317d0f..2ee4e80 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -72,12 +72,6 @@ private void processLoop() { continue; } - if(!tableAdded && - PixelsSinkConfigFactory.getInstance().getSinkMonitorFreshnessLevel().equals("embed")) - { - tableAdded = true; - FreshnessClient.getInstance().addMonitoredTable(event.getTable()); - } pixelsSinkWriter.writeRow(event); } LOGGER.info("Processor thread exited"); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java index 7b98718..ab09b74 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java @@ -22,6 +22,7 @@ import com.google.protobuf.InvalidProtocolBufferException; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; import io.pixelsdb.pixels.sink.exception.SinkException; @@ -29,13 +30,28 @@ import java.nio.ByteBuffer; import java.util.logging.Logger; import io.pixelsdb.pixels.core.utils.Pair; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; +import io.pixelsdb.pixels.sink.util.DataTransform; + +import javax.xml.crypto.Data; + public class TableEventStorageLoopProvider extends TableEventProvider { private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); + private final boolean freshness_embed; + protected TableEventStorageLoopProvider() { super(); + String sinkMonitorFreshnessLevel = PixelsSinkConfigFactory.getInstance().getSinkMonitorFreshnessLevel(); + if(sinkMonitorFreshnessLevel.equals("embed")) + { + freshness_embed = true; + } else + { + freshness_embed = false; + } } @Override @@ -49,9 +65,14 @@ RowChangeEvent convertToTargetRecord(T record) SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); SinkProto.RowRecord.Builder rowRecordBuilder = rowRecord.toBuilder(); + if(freshness_embed) + { + DataTransform.updateRecordTimestamp(rowRecordBuilder, System.currentTimeMillis() * 1000); + FreshnessClient.getInstance().addMonitoredTable(rowRecord.getSource().getTable()); + } SinkProto.TransactionInfo.Builder transactionBuilder = rowRecordBuilder.getTransactionBuilder(); String id = transactionBuilder.getId(); - transactionBuilder.setId(id + "_" + Integer.toString(loopId)); + transactionBuilder.setId(id + "_" + loopId); rowRecordBuilder.setTransaction(transactionBuilder); return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecordBuilder.build()); } catch (InvalidProtocolBufferException | SinkException e) diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java index b0505ce..f0cd58f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java @@ -92,7 +92,6 @@ protected AbstractSinkStorageSource() protected void handleTransactionSourceRecord(ByteBuffer record, Integer loopId) { - sourceRateLimiter.acquire(1); transactionEventProvider.putTransRawEvent(new Pair<>(record, loopId)); } @@ -154,8 +153,11 @@ public void start() k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE)); // Put future in queue + if(protoType.equals(ProtoType.ROW)) + { + sourceRateLimiter.acquire(1); + } queue.put(new Pair<>(valueFuture, loopId)); - // Start consumer thread if not exists consumerThreads.computeIfAbsent(key, k -> { diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java index dc41b50..11b24b8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java @@ -20,6 +20,7 @@ package io.pixelsdb.pixels.sink.util; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Semaphore; @@ -128,4 +129,9 @@ public V compute(K key, BiFunction remappingF } } } + + public Set keySet() + { + return map.keySet(); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java index 595675e..661f4c7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -30,78 +30,119 @@ public class DataTransform { - public static ByteString longToByteString(long value) { + private static ByteString longToByteString(long value) + { byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); return ByteString.copyFrom(bytes); } - public static void updateTimeStamp(List updateData, long txStartTime) { + @Deprecated + public static void updateTimeStamp(List updateData, long txStartTime) + { ByteString timestampBytes = longToByteString(txStartTime); - for (RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder : updateData) { + for (RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder : updateData) + { int insertDataCount = tableUpdateDataBuilder.getInsertDataCount(); - for (int i = 0; i < insertDataCount; i++) { + for (int i = 0; i < insertDataCount; i++) + { RetinaProto.InsertData.Builder insertBuilder = tableUpdateDataBuilder.getInsertDataBuilder(i); int colValueCount = insertBuilder.getColValuesCount(); - if (colValueCount > 0) { + if (colValueCount > 0) + { insertBuilder.setColValues(colValueCount - 1, timestampBytes); } } int updateDataCount = tableUpdateDataBuilder.getUpdateDataCount(); - for (int i = 0; i < updateDataCount; i++) { + for (int i = 0; i < updateDataCount; i++) + { RetinaProto.UpdateData.Builder updateBuilder = tableUpdateDataBuilder.getUpdateDataBuilder(i); int colValueCount = updateBuilder.getColValuesCount(); - if (colValueCount > 0) { + if (colValueCount > 0) + { updateBuilder.setColValues(colValueCount - 1, timestampBytes); } } } } - /** - * 遍历 RowRecord 列表,为每个记录的 'after' 镜像的最后一列更新时间戳。 - * 由于 RowRecord 是不可变的,此方法会返回一个包含已修改记录的新列表。 - * - * @param records 原始的 RowRecord 列表。 - * @param timestamp 要设置的时间戳 (long 类型)。 - * @return 包含更新后时间戳的 RowRecord 新列表。 - */ - public static List updateRecordTimestamp(List records, long timestamp) { - // 处理空或 null 列表的边界情况 + + + + public static List updateRecordTimestamp(List records, long timestamp) + { if (records == null || records.isEmpty()) { return records; } - // 1. 一次性将 long 转换为 ByteString,提高效率 - ByteString timestampBytes = longToByteString(timestamp); - SinkProto.ColumnValue timestampColumn = SinkProto.ColumnValue.newBuilder().setValue(timestampBytes).build(); - // 2. 创建一个新列表来存储修改后的记录 + SinkProto.ColumnValue timestampColumn = getTimestampColumn(timestamp); List updatedRecords = new ArrayList<>(records.size()); - // 3. 遍历所有记录 for (SinkProto.RowRecord record : records) { - SinkProto.RowRecord.Builder recordBuilder = record.toBuilder(); - // 4. 只处理包含 'after' 镜像的操作类型 (INSERT, UPDATE, SNAPSHOT) - switch (record.getOp()) { - case INSERT: - case UPDATE: - case SNAPSHOT: - if (recordBuilder.hasAfter()) { - SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); - int colCount = afterBuilder.getValuesCount(); - if (colCount > 0) { - // 5. 设置最后一列的值 - afterBuilder.setValues(colCount - 1, timestampColumn); - } - } - break; - case DELETE: - default: - // 对于 DELETE 或其他未知操作,我们不修改记录 - break; - } - // 6. 将构建好的记录(无论是否修改)添加到新列表中 - updatedRecords.add(recordBuilder.build()); + + updatedRecords.add(updateRecordTimestamp(record, timestampColumn)); } return updatedRecords; } + + private static SinkProto.ColumnValue getTimestampColumn(long timestamp) + { + ByteString timestampBytes = longToByteString(timestamp); + return SinkProto.ColumnValue.newBuilder().setValue(timestampBytes).build(); + } + + public static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord record, long timestamp) + { + if (record == null) { + return null; + } + SinkProto.ColumnValue timestampColumn = getTimestampColumn(timestamp); + return updateRecordTimestamp(record, timestampColumn); + } + + public static void updateRecordTimestamp(SinkProto.RowRecord.Builder recordBuilder, long timestamp) + { + switch (recordBuilder.getOp()) { + case INSERT: + case UPDATE: + case SNAPSHOT: + if (recordBuilder.hasAfter()) { + SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); + int colCount = afterBuilder.getValuesCount(); + if (colCount > 0) { + afterBuilder.setValues(colCount - 1, getTimestampColumn(timestamp)); + } + } + break; + case DELETE: + default: + break; + } + } + + private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord.Builder recordBuilder, SinkProto.ColumnValue timestampColumn) + { + switch (recordBuilder.getOp()) { + case INSERT: + case UPDATE: + case SNAPSHOT: + if (recordBuilder.hasAfter()) { + SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); + int colCount = afterBuilder.getValuesCount(); + if (colCount > 0) { + afterBuilder.setValues(colCount - 1, timestampColumn); + } + } + break; + case DELETE: + default: + break; + } + return recordBuilder.build(); + } + + private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord record, SinkProto.ColumnValue timestampColumn) + { + SinkProto.RowRecord.Builder recordBuilder = record.toBuilder(); + return updateRecordTimestamp(recordBuilder, timestampColumn); + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index bcb5e9d..8849095 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -552,14 +552,15 @@ public void logPerformance() LOGGER.info( "Performance report: +{} rows (+{}/s), +{} transactions (+{}/s), +{} debezium (+{}/s)" + ", +{} serdRows (+{}/s), +{} serdTxs (+{}/s)" + - " in {} ms\t activeTxNum: {}", + " in {} ms\t activeTxNum: {} min Tx: {}", deltaRows, String.format("%.2f", rowOips), deltaTxns, String.format("%.2f", txnOips), deltaDebezium, String.format("%.2f", dbOips), deltaSerdRows, String.format("%.2f", serdRowsOips), deltaSerdTxs, String.format("%.2f", serdTxsOips), monitorReportInterval, - sinkContextManager.getActiveTxnsNum() + sinkContextManager.getActiveTxnsNum(), + sinkContextManager.findMinActiveTx() ); LOGGER.info( diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java index 49e15fc..75e293f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java @@ -79,17 +79,10 @@ public void pollEvents(SinkProto.PollRequest request, StreamObserver updatedRecords = updatedRecords = DataTransform.updateRecordTimestamp( - records, - System.currentTimeMillis() - ); - - if (updatedRecords != null && !updatedRecords.isEmpty()) { - responseBuilder.addAllRecords(updatedRecords); - this.flushRateLimiter.acquire(updatedRecords.size()); - } + responseBuilder.addAllRecords(records); + this.flushRateLimiter.acquire(records.size()); } responseObserver.onNext(responseBuilder.build()); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index ef19264..728b9c2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -21,6 +21,7 @@ package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.transaction.TransContext; +import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; @@ -29,6 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.LocalDateTime; import java.util.Map; import java.util.Queue; import java.util.concurrent.CompletableFuture; @@ -63,9 +65,9 @@ public class SinkContext @Getter Map tableCounters = new ConcurrentHashMap<>(); @Getter + @Setter Queue orphanEvent = new ConcurrentLinkedQueue<>(); @Getter - @Setter private TransContext pixelsTransCtx; @Setter @Getter @@ -75,6 +77,9 @@ public class SinkContext @Setter private volatile Long startTime = null; + + private final Queue> recordTimes = new ConcurrentLinkedQueue<>(); + @Getter @Setter SinkProto.TransactionMetadata endTx; @@ -96,6 +101,20 @@ void updateCounter(String table) updateCounter(table, 1L); } + public void setPixelsTransCtx(TransContext pixelsTransCtx) + { + if(this.pixelsTransCtx != null) + { + throw new IllegalStateException("Pixels Trans Context Already Set"); + } + this.pixelsTransCtx = pixelsTransCtx; + } + + public void recordTimestamp(String table, LocalDateTime timestamp) + { + recordTimes.offer(new Pair<>(table, timestamp)); + } + public void updateCounter(String table, long count) { tableCounterLock.lock(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index c556a46..39d52e3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -21,7 +21,7 @@ package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.TransException; -import io.pixelsdb.pixels.common.transaction.TransService; +import io.pixelsdb.pixels.common.transaction.TransContext; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -32,6 +32,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Comparator; +import java.util.Optional; import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -114,6 +116,8 @@ protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean can protected void startTransSync(String sourceTxId) { + LOGGER.trace("Start trans {}", sourceTxId); + TransContext pixelsTransContext = transactionProxy.getNewTransContext(sourceTxId); activeTxContexts.compute( sourceTxId, (k, oldCtx) -> @@ -121,7 +125,7 @@ protected void startTransSync(String sourceTxId) if (oldCtx == null) { LOGGER.trace("Start trans {} without buffered events", sourceTxId); - return new SinkContext(sourceTxId, transactionProxy.getNewTransContext()); + return new SinkContext(sourceTxId, pixelsTransContext); } else { oldCtx.getLock().lock(); @@ -132,7 +136,8 @@ protected void startTransSync(String sourceTxId) LOGGER.warn("Previous tx {} has been released, maybe due to loop process", sourceTxId); oldCtx.tableCounters = new ConcurrentHashMap<>(); } - oldCtx.setPixelsTransCtx(transactionProxy.getNewTransContext()); + LOGGER.trace("Start trans with buffered events {}", sourceTxId); + oldCtx.setPixelsTransCtx(pixelsTransContext); handleOrphanEvents(oldCtx); oldCtx.getCond().signalAll(); } catch (SinkException e) @@ -181,7 +186,7 @@ void endTransaction(SinkContext ctx) boolean failed = ctx.isFailed(); if (!failed) { - LOGGER.trace("Committed transaction: {}", txId); + LOGGER.trace("Committed transaction: {}, Pixels Trans is {}", txId, ctx.getPixelsTransCtx().getTransId()); switch (commitMethod) { case Sync -> @@ -213,7 +218,7 @@ void endTransaction(SinkContext ctx) private void handleOrphanEvents(SinkContext ctx) throws SinkException { Queue buffered = ctx.getOrphanEvent(); - + ctx.setOrphanEvent(null); if (buffered != null) { LOGGER.trace("Handle Orphan Events in {}", ctx.sourceTxId); @@ -312,6 +317,33 @@ public int getActiveTxnsNum() return activeTxContexts.size(); } + public String findMinActiveTx() + { + Comparator customComparator = (key1, key2) -> { + try { + String[] parts1 = key1.split("_"); + int int1 = Integer.parseInt(parts1[0]); + int loopId1 = Integer.parseInt(parts1[1]); + + String[] parts2 = key2.split("_"); + int int2 = Integer.parseInt(parts2[0]); + int loopId2 = Integer.parseInt(parts2[1]); + + int loopIdComparison = Integer.compare(loopId1, loopId2); + if (loopIdComparison != 0) { + return loopIdComparison; + } + return Integer.compare(int1, int2); + } catch (Exception e) { + System.err.println("Key format error for comparison: " + e.getMessage()); + return 0; + } + }; + + Optional min = activeTxContexts.keySet().stream().min(customComparator); + return min.orElse("None"); + } + private enum CommitMethod { Sync, diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 0894c62..0d92fa9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -30,6 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.LocalDateTime; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -120,21 +121,14 @@ public void flush() } } - flushRateLimiter.acquire(batch.size()); + // flushRateLimiter.acquire(batch.size()); long txStartTime = System.currentTimeMillis(); // if(freshnessLevel.equals("embed")) - if (true) - { - long freshness_ts = txStartTime * 1000; - FreshnessClient.getInstance().addMonitoredTable(tableName); - DataTransform.updateTimeStamp(tableUpdateDataBuilderList, freshness_ts); - } - - -// for(String writeTxId: txIds) // { -// sinkContextManager.getSinkContext(writeTxId).setCurrStartTime(); +// long freshness_ts = txStartTime * 1000; +// FreshnessClient.getInstance().addMonitoredTable(tableName); +// DataTransform.updateTimeStamp(tableUpdateDataBuilderList, freshness_ts); // } List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); @@ -191,6 +185,7 @@ private void updateCtxCounters(List txIds, List fullTableName, L try { sinkContext.tableCounterLock.lock(); + sinkContext.recordTimestamp(fullTableName.get(i), LocalDateTime.now()); sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); if(sinkContext.isCompleted()) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java index 9671c16..d21f875 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java @@ -26,6 +26,9 @@ import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.freshness.FreshnessClient; import io.pixelsdb.pixels.sink.util.DataTransform; +import lombok.Getter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.LinkedList; @@ -35,6 +38,8 @@ public class TableSingleRecordWriter extends TableCrossTxWriter { + @Getter + private final Logger LOGGER = LoggerFactory.getLogger(TableSingleRecordWriter.class); private final TransactionProxy transactionProxy; public TableSingleRecordWriter(String t, int bucketId) @@ -82,17 +87,9 @@ public void flush() tableUpdateDataBuilderList.add(builder); } - flushRateLimiter.acquire(batch.size()); + // flushRateLimiter.acquire(batch.size()); long txStartTime = System.currentTimeMillis(); -// if(freshnessLevel.equals("embed")) - if (true) - { - long freshness_ts = txStartTime * 1000; - FreshnessClient.getInstance().addMonitoredTable(tableName); - DataTransform.updateTimeStamp(tableUpdateDataBuilderList, freshness_ts); - } - List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index c1f0fae..920706d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -37,6 +37,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; /** @@ -50,6 +51,7 @@ public abstract class TableWriter protected final RetinaServiceProxy delegate; // physical writer protected final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + protected final ScheduledExecutorService logScheduler = Executors.newScheduledThreadPool(1); protected final ReentrantLock bufferLock = new ReentrantLock(); protected final String tableName; protected final long flushInterval; @@ -65,6 +67,8 @@ public abstract class TableWriter protected PixelsSinkConfig config; protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); protected TransactionMode transactionMode; + private final AtomicInteger counter = new AtomicInteger(); + protected TableWriter(String tableName, int bucketId) { this.config = PixelsSinkConfigFactory.getInstance(); @@ -75,6 +79,42 @@ protected TableWriter(String tableName, int bucketId) this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); this.delegate = new RetinaServiceProxy(bucketId); this.transactionMode = config.getTransactionMode(); + + if(this.config.isMonitorReportEnabled()) + { + long interval = this.config.getMonitorReportInterval(); + Runnable monitorTask = writerInfoTask(tableName); + logScheduler.scheduleAtFixedRate( + monitorTask, + 0, + interval, + TimeUnit.MILLISECONDS + ); + } + } + + private Runnable writerInfoTask(String tableName) + { + final AtomicInteger reportId = new AtomicInteger(); + final AtomicInteger lastRunCounter = new AtomicInteger(); + Runnable monitorTask = () -> { + String firstTx = "none"; + RowChangeEvent firstEvent = null; + int len = 0; + bufferLock.lock(); + len = buffer.size(); + if (!buffer.isEmpty()) { + firstEvent = buffer.get(0); + } + bufferLock.unlock(); + if(firstEvent != null) + { + firstTx = firstEvent.getTransaction().getId(); + int count = counter.get(); + getLOGGER().debug("{} Writer {}: Tx Now is {}. Buffer Len is {}. Total Count {}", reportId.incrementAndGet(), tableName, firstTx, len, count); + } + }; + return monitorTask; } /** @@ -140,6 +180,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) { fullTableName = event.getFullTableName(); } + counter.incrementAndGet(); buffer.add(event); // Reset scheduled flush: cancel old one and reschedule @@ -186,6 +227,7 @@ public boolean write(RowChangeEvent event, SinkContext ctx) public void close() { scheduler.shutdown(); + logScheduler.shutdown(); try { scheduler.awaitTermination(5, TimeUnit.SECONDS); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index db55f88..1a94514 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -121,19 +121,28 @@ private void requestTransactions() } } + @Deprecated public TransContext getNewTransContext() + { + return getNewTransContext("None"); + } + + public TransContext getNewTransContext(String txId) { beginCount.incrementAndGet(); if(true) { try { - return transService.beginTrans(false); + TransContext transContext = transService.beginTrans(false); + LOGGER.trace("{} begin {}", txId, transContext.getTransId()); + return transContext; } catch (TransException e) { throw null; } } + TransContext ctx = transContextQueue.poll(); if (ctx != null) { From 0b8e4447fdebd9de9a27ea6e7b829b792c112185 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 9 Dec 2025 09:15:31 +0000 Subject: [PATCH 43/53] Verbose Freshness --- conf/pixels-sink.aws.properties | 2 + .../pixels/sink/config/PixelsSinkConfig.java | 7 + .../sink/freshness/FreshnessClient.java | 238 ++++++++++-------- .../sink/freshness/FreshnessHistory.java | 58 +++++ .../{util => freshness}/OneSecondAverage.java | 2 +- .../pixels/sink/util/MetricsFacade.java | 45 +++- 6 files changed, 247 insertions(+), 105 deletions(-) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java rename src/main/java/io/pixelsdb/pixels/sink/{util => freshness}/OneSecondAverage.java (98%) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index c236daf..7189c9f 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -18,6 +18,8 @@ trino.parallel=5 # row or txn or embed sink.monitor.freshness.level=embed sink.monitor.freshness.embed.warmup=10 +sink.monitor.freshness.verbose=true + sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index ff4fd29..581e09f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -181,6 +181,10 @@ public class PixelsSinkConfig private String sinkMonitorFreshnessLevel; @ConfigKey(value = "sink.monitor.freshness.embed.warmup", defaultValue = "10") private Integer sinkMonitorFreshnessEmbedWarmupSeconds; + + @ConfigKey(value = "sink.monitor.freshness.verbose", defaultValue = "false") + private boolean sinkMonitorFreshnessVerbose; + @ConfigKey(value = "trino.url") private String trinoUrl; @@ -190,6 +194,9 @@ public class PixelsSinkConfig @ConfigKey(value = "trino.password") private String trinoPassword; + @ConfigKey(value = "trino.parallel", defaultValue = "1") + private int trinoParallel; + public PixelsSinkConfig(String configFilePath) throws IOException { this.config = ConfigFactory.Instance(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index ea22b38..ecdece3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.freshness; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -30,31 +30,32 @@ import java.sql.*; import java.util.*; import java.util.Date; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.*; /** * FreshnessClient is responsible for monitoring data freshness by periodically * querying the maximum timestamp from a set of dynamically configured tables via Trino JDBC. */ -public class FreshnessClient { +public class FreshnessClient +{ private static final Logger LOGGER = LoggerFactory.getLogger(FreshnessClient.class); - + private static final int QUERY_INTERVAL_SECONDS = 1; + private static volatile FreshnessClient instance; // Configuration parameters (should ideally be loaded from a config file) private final String trinoJdbcUrl; private final String trinoUser; private final String trinoPassword; - + private final int maxConcurrentQueries; + private final Semaphore queryPermits; + private final ThreadPoolExecutor connectionExecutor; // Key modification: Use a thread-safe Set to maintain the list of tables to monitor dynamically. private final Set monitoredTables; - private static final int QUERY_INTERVAL_SECONDS = 1; - private Connection connection; private final ScheduledExecutorService scheduler; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private static volatile FreshnessClient instance; private final int warmUpSeconds; - private FreshnessClient() { + + private FreshnessClient() + { // Initializes the set with thread safety wrapper this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); @@ -63,20 +64,43 @@ private FreshnessClient() { this.trinoJdbcUrl = config.getTrinoUrl(); this.trinoPassword = config.getTrinoPassword(); this.warmUpSeconds = config.getSinkMonitorFreshnessEmbedWarmupSeconds(); + this.maxConcurrentQueries = config.getTrinoParallel(); + this.queryPermits = new Semaphore(maxConcurrentQueries); // Initializes a single-threaded scheduler for executing freshness queries - this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> + { Thread t = Executors.defaultThreadFactory().newThread(r); t.setName("Freshness-Client-Scheduler"); t.setDaemon(true); return t; }); + + this.connectionExecutor = new ThreadPoolExecutor( + maxConcurrentQueries, + maxConcurrentQueries, + 60L, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + r -> + { + Thread t = Executors.defaultThreadFactory().newThread(r); + t.setName("Freshness-Query-Worker"); + t.setDaemon(true); + return t; + } + ); + this.connectionExecutor.allowCoreThreadTimeOut(true); } - public static FreshnessClient getInstance() { - if (instance == null) { + public static FreshnessClient getInstance() + { + if (instance == null) + { // First check: Reduces synchronization overhead once the instance is created - synchronized (FreshnessClient.class) { - if (instance == null) { + synchronized (FreshnessClient.class) + { + if (instance == null) + { // Second check: Only one thread proceeds to create the instance instance = new FreshnessClient(); } @@ -85,17 +109,8 @@ public static FreshnessClient getInstance() { return instance; } - // ------------------------------------------------------------------------------------------------- - // Connection Management - // ------------------------------------------------------------------------------------------------- - - /** - * Establishes a new JDBC connection to the Trino coordinator. - * @throws SQLException if the connection cannot be established. - */ - private void establishConnection() throws SQLException + private Connection createNewConnection() throws SQLException { - LOGGER.info("Attempting to connect to Trino via JDBC: {}", trinoJdbcUrl); try { Class.forName("io.trino.jdbc.TrinoDriver"); @@ -103,33 +118,20 @@ private void establishConnection() throws SQLException { throw new SQLException(e); } - this.connection = DriverManager.getConnection(trinoJdbcUrl, trinoUser, null); - LOGGER.info("Trino connection established successfully."); + return DriverManager.getConnection(trinoJdbcUrl, trinoUser, null); } - /** - * Ensures the current connection is valid, re-establishing it if necessary (closed, null, or invalid). - * @throws SQLException if connection cannot be re-established. - */ - private void ensureConnectionValid() throws SQLException { - if (connection == null || connection.isClosed() || !connection.isValid(5)) { - closeConnection(); - establishConnection(); - } - } - - /** - * Safely closes the current JDBC connection. - */ - private void closeConnection() { - if (connection != null) { - try { - connection.close(); - LOGGER.info("Trino connection closed."); - } catch (SQLException e) { + private void closeConnection(Connection conn) + { + if (conn != null) + { + try + { + conn.close(); + } catch (SQLException e) + { LOGGER.warn("Error closing Trino connection.", e); } - connection = null; } } @@ -140,10 +142,13 @@ private void closeConnection() { /** * Adds a table name to the monitoring list. * This method can be called by external components (e.g., config trigger). + * * @param tableName The name of the table to add. */ - public void addMonitoredTable(String tableName) { - if (tableName == null || tableName.trim().isEmpty()) { + public void addMonitoredTable(String tableName) + { + if (tableName == null || tableName.trim().isEmpty()) + { LOGGER.warn("Attempted to add null or empty table name to freshness monitor."); return; } @@ -152,12 +157,16 @@ public void addMonitoredTable(String tableName) { /** * Removes a table name from the monitoring list. + * * @param tableName The name of the table to remove. */ - public void removeMonitoredTable(String tableName) { - if (monitoredTables.remove(tableName)) { + public void removeMonitoredTable(String tableName) + { + if (monitoredTables.remove(tableName)) + { LOGGER.info("Table '{}' removed from freshness monitor list.", tableName); - } else { + } else + { LOGGER.debug("Table '{}' was not found in the freshness monitor list.", tableName); } } @@ -169,52 +178,76 @@ public void removeMonitoredTable(String tableName) { /** * Starts the scheduled freshness monitoring task. */ - public void start() { - try { - ensureConnectionValid(); - LOGGER.info("Starting Freshness Client, querying every {} seconds.", QUERY_INTERVAL_SECONDS); - - scheduler.scheduleAtFixedRate(this::queryAndCalculateFreshness, - warmUpSeconds, - QUERY_INTERVAL_SECONDS, - TimeUnit.SECONDS); - } catch (SQLException e) { - LOGGER.error("Failed to establish initial Trino connection. Freshness Client will not start.", e); - } + public void start() + { + LOGGER.info("Starting Freshness Client, querying every {} seconds.", QUERY_INTERVAL_SECONDS); + scheduler.scheduleAtFixedRate(this::submitQueryTask, + warmUpSeconds, + QUERY_INTERVAL_SECONDS, + TimeUnit.SECONDS); } /** * Stops the scheduled task and closes the JDBC connection. */ - public void stop() { + public void stop() + { LOGGER.info("Stopping Freshness Client."); scheduler.shutdownNow(); - closeConnection(); + connectionExecutor.shutdownNow(); } - /** - * The core scheduled task: queries max(freshness_ts) for all monitored tables - * and calculates the freshness metric. - */ - void queryAndCalculateFreshness() { - try { - ensureConnectionValid(); - } catch (SQLException e) { - LOGGER.error("Connection failed during scheduled run. Skipping this freshness cycle.", e); + private void submitQueryTask() + { + if (monitoredTables.isEmpty()) + { + LOGGER.debug("No tables configured for freshness monitoring. Skipping submission cycle."); return; } - // Take a snapshot of the tables to monitor for this cycle. - // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. - Set tablesSnapshot = new HashSet<>(monitoredTables); - if (tablesSnapshot.isEmpty()) { - LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); + if (!queryPermits.tryAcquire()) + { + LOGGER.debug("Max concurrent queries ({}) reached. Skipping query submission this cycle.", maxConcurrentQueries); return; } + try + { + connectionExecutor.submit(this::queryAndCalculateFreshness); + } catch (RejectedExecutionException e) + { + queryPermits.release(); + LOGGER.error("Query task rejected by executor. Max concurrent queries may be too low or service is stopping.", e); + } catch (Exception e) + { + queryPermits.release(); + LOGGER.error("Unknown error during task submission.", e); + } + } + + /** + * The core scheduled task: queries max(freshness_ts) for all monitored tables + * and calculates the freshness metric. + */ + void queryAndCalculateFreshness() + { + Connection conn = null; + + String tableName; - try { + try + { + + // Take a snapshot of the tables to monitor for this cycle. + // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. + Set tablesSnapshot = new HashSet<>(monitoredTables); + if (tablesSnapshot.isEmpty()) + { + LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); + return; + } + monitoredTables.clear(); List tableList = new ArrayList<>(tablesSnapshot); Random random = new Random(); @@ -223,46 +256,55 @@ void queryAndCalculateFreshness() { tableName = tableList.get(randomIndex); LOGGER.debug("Randomly selected table for this cycle: {}", tableName); - - } catch (Exception e) { - LOGGER.error("Error selecting a random table from the monitor list.", e); - return; - } - + conn = createNewConnection(); // Timestamp when the query is sent (t_send) long tSendMillis = System.currentTimeMillis(); - String tSendMillisStr = DateUtil.convertDateToString(new Date(tSendMillis)); + String tSendMillisStr = DateUtil.convertDateToString(new Date(tSendMillis)); // Query to find the latest timestamp in the table // Assumes 'freshness_ts' is a long-type epoch timestamp (milliseconds) String query = String.format("SELECT max(freshness_ts) FROM %s WHERE freshness_ts < TIMESTAMP '%s'", tableName, tSendMillisStr); - try (Statement statement = connection.createStatement(); - ResultSet rs = statement.executeQuery(query)) { + try (Statement statement = conn.createStatement(); + ResultSet rs = statement.executeQuery(query)) + { Timestamp maxFreshnessTs = null; - if (rs.next()) { + if (rs.next()) + { // Read the maximum timestamp value maxFreshnessTs = rs.getTimestamp(1); } - if (maxFreshnessTs != null) { + if (maxFreshnessTs != null) + { // Freshness = t_send - data_write_time (maxFreshnessTs) // Result is in milliseconds long freshnessMillis = tSendMillis - maxFreshnessTs.getTime(); metricsFacade.recordFreshness(freshnessMillis); - } else { + } else + { LOGGER.warn("Table {} returned null or zero max(freshness_ts). Skipping freshness calculation.", tableName); } - } catch (SQLException e) { + } catch (SQLException e) + { // Handle database errors (e.g., table not found, query syntax error) LOGGER.error("Failed to execute query for table {}: {}", tableName, e.getMessage()); - } catch (Exception e) { + } catch (Exception e) + { // Catch potential runtime errors (e.g., in MetricsFacade) LOGGER.error("Error calculating or recording freshness for table {}.", tableName, e); } - monitoredTables.clear(); + } catch (Exception e) + { + LOGGER.error("Error selecting a random table from the monitor list.", e); + } finally + { + closeConnection(conn); + queryPermits.release(); + } + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java new file mode 100644 index 0000000..004c471 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java @@ -0,0 +1,58 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +package io.pixelsdb.pixels.sink.freshness; + + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; + +public class FreshnessHistory { + private final ConcurrentLinkedQueue history = new ConcurrentLinkedQueue<>(); + + public record Record(long timestamp, double value) + { + + @Override + public String toString() + { + return timestamp + "," + value; + } + } + + public void record(double freshnessMill) { + history.offer(new Record(System.currentTimeMillis(), freshnessMill)); + } + + public List pollAll() + { + if (history.isEmpty()) { + return Collections.emptyList(); + } + List records = new ArrayList<>(); + Record record; + while ((record = history.poll()) != null) { + records.add(record); + } + return records; + } +} \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java rename to src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java index 555ddf5..01123e8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/OneSecondAverage.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.util; +package io.pixelsdb.pixels.sink.freshness; import java.util.ArrayDeque; import java.util.Deque; diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 8849095..7f2b0fb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -25,6 +25,8 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.freshness.FreshnessHistory; +import io.pixelsdb.pixels.sink.freshness.OneSecondAverage; import io.pixelsdb.pixels.sink.writer.retina.SinkContextManager; import io.prometheus.client.Counter; import io.prometheus.client.Histogram; @@ -39,6 +41,7 @@ import java.nio.ByteBuffer; import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; +import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; public class MetricsFacade @@ -71,6 +74,8 @@ public class MetricsFacade private final SynchronizedDescriptiveStatistics freshness; private final SynchronizedDescriptiveStatistics rowChangeSpeed; private final OneSecondAverage freshnessAvg; + private final Boolean freshnessVerbose; + private final FreshnessHistory freshnessHistory; private final String monitorReportPath; private final String freshnessReportPath; @@ -227,6 +232,15 @@ private MetricsFacade(boolean enabled) freshnessReportInterval = config.getFreshnessReportInterval(); freshnessReportPath = config.getMonitorFreshnessReportFile(); freshnessAvg = new OneSecondAverage(freshnessReportInterval); + freshnessVerbose = config.isSinkMonitorFreshnessVerbose(); + if(freshnessVerbose) + { + freshnessHistory = new FreshnessHistory(); + } else + { + freshnessHistory = null; + } + monitorReportEnabled = config.isMonitorReportEnabled(); monitorReportInterval = config.getMonitorReportInterval(); @@ -415,6 +429,11 @@ public void recordFreshness(double freshnessMill) { freshnessAvg.record(freshnessMill); } + + if(freshnessVerbose) + { + freshnessHistory.record(freshnessMill); + } } public void recordPrimaryKeyUpdateDistribution(String table, ByteString pkValue) { @@ -496,14 +515,28 @@ public void runFreshness() Thread.sleep(freshnessReportInterval); try (FileWriter fw = new FileWriter(freshnessReportPath, true)) { - long now = System.currentTimeMillis(); - double avg = freshnessAvg.getWindowAverage(); - if(Double.isNaN(avg)) + if(freshnessVerbose) + { + List detailedRecords = freshnessHistory.pollAll(); + if (!detailedRecords.isEmpty()) + { + for (FreshnessHistory.Record record : detailedRecords) + { + fw.write(record.toString() + "\n"); + } + fw.flush(); + } + } else { - continue; + long now = System.currentTimeMillis(); + double avg = freshnessAvg.getWindowAverage(); + if(Double.isNaN(avg)) + { + continue; + } + fw.write(now + "," + avg + "\n"); + fw.flush(); } - fw.write(now + "," + avg + "\n"); - fw.flush(); } catch (IOException e) { LOGGER.warn("Failed to write perf metrics: " + e.getMessage()); From a07b759aab09fc6f6d377bb9f2fc7d3f5043c39e Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 11 Dec 2025 10:48:06 +0000 Subject: [PATCH 44/53] Embed Freshness --- conf/pixels-sink.aws.properties | 16 +++--- perf_freshness.py | 30 ++++++----- .../pixels/sink/config/PixelsSinkConfig.java | 3 ++ .../sink/freshness/FreshnessClient.java | 3 +- .../provider/TableEventKafkaProvider.java | 9 +--- .../TableEventStorageLoopProvider.java | 11 ++-- .../pixels/sink/util/DataTransform.java | 6 +++ .../pixels/sink/util/MetricsFacade.java | 50 +++++++++---------- .../sink/writer/retina/SinkContext.java | 2 + .../writer/retina/SinkContextManager.java | 15 ++++-- .../retina/TableSingleRecordWriter.java | 12 +++++ .../sink/writer/retina/TableWriter.java | 13 +++-- 12 files changed, 106 insertions(+), 64 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 7189c9f..fd80e14 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,17 +1,17 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=10000 +sink.datasource.rate.limit=20000 # Sink Config: retina | csv | proto | flink | none sink.mode=retina sink.retina.client=8 ## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature sink.trans.mode=batch sink.monitor.report.enable=true -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/i7i_10k_dec.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/freshness_i7i_10k_dec.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/i7i_2k_batchtest_dec_rate.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/i7i_2k_batchtest_dec_freshness_2.csv # trino for freshness query -trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x +trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x trino.user=pixels trino.password=password trino.parallel=5 @@ -19,7 +19,7 @@ trino.parallel=5 sink.monitor.freshness.level=embed sink.monitor.freshness.embed.warmup=10 sink.monitor.freshness.verbose=true - +sink.monitor.freshness.timestamp=true sink.storage.loop=true # Kafka Config bootstrap.servers=realtime-kafka-2:29092 @@ -42,8 +42,8 @@ sink.retina.mode=stream sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 -sink.flush.interval.ms=200 -sink.flush.batch.size=50 +sink.flush.interval.ms=50 +sink.flush.batch.size=10 sink.max.retries=3 ## writer commit # sync or async @@ -67,7 +67,7 @@ sink.trans.batch.size=100 # Sink Metrics sink.monitor.enable=true -sink.monitor.port=9465 +sink.monitor.port=9464 sink.monitor.report.interval=10000 sink.monitor.freshness.interval=1000 diff --git a/perf_freshness.py b/perf_freshness.py index 8bf183d..51b0904 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -5,19 +5,23 @@ ########################################## # 配置 CSV 文件 和 标签 ########################################## +# csv_files = { +# "10k": "tmp/freshness10k_5.csv", +# "20k": "tmp/freshness20k_5.csv", +# # "30k": "tmp/freshness30k_2.csv", +# "40k": "tmp/freshness40k_5.csv", +# "50k": "tmp/freshness50k_5.csv", +# "60k": "tmp/freshness60k_5.csv" +# } csv_files = { - "10k": "tmp/freshness10k_5.csv", - "20k": "tmp/freshness20k_5.csv", - # "30k": "tmp/freshness30k_2.csv", - "40k": "tmp/freshness40k_5.csv", - "50k": "tmp/freshness50k_5.csv", - "60k": "tmp/freshness60k_5.csv" + "Query Transaction": "tmp/i7i_2k_dec_freshness.csv", + "Query Record": "tmp/i7i_2k_record_dec_freshness.csv", + "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv" } - -MAX_SECONDS = 1800 # 截取前多少秒的数据 -SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) -BIN_SECONDS = 60 # 平均窗口(秒) - +MAX_SECONDS = 500 # 截取前多少秒的数据 +SKIP_SECONDS = 20 # 跳过前多少秒的数据(可调) +BIN_SECONDS = 10 # 平均窗口(秒) +MAX_FRESHNESS = 5000 ########################################## # 加载并处理数据 ########################################## @@ -35,6 +39,8 @@ # 跳过前 SKIP_SECONDS 秒 df = df[df["sec"] >= SKIP_SECONDS] + df = df[df["freshness"] <= MAX_FRESHNESS] + # 重新计算时间(所有曲线从 0 秒开始对齐) t_new0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t_new0).dt.total_seconds() @@ -61,7 +67,7 @@ plt.xlabel("Time (sec)") plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)") -plt.yscale("log") +# plt.yscale("log") plt.title( f"Freshness Over Time ({BIN_SECONDS}-Second Avg, " f"Skip {SKIP_SECONDS}s, First {MAX_SECONDS}s)" diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 581e09f..52c9f0b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -185,6 +185,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.freshness.verbose", defaultValue = "false") private boolean sinkMonitorFreshnessVerbose; + @ConfigKey(value = "sink.monitor.freshness.timestamp", defaultValue = "false") + private boolean sinkMonitorFreshnessTimestamp; + @ConfigKey(value = "trino.url") private String trinoUrl; diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index ecdece3..cc4add5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -281,8 +281,7 @@ void queryAndCalculateFreshness() // Freshness = t_send - data_write_time (maxFreshnessTs) // Result is in milliseconds long freshnessMillis = tSendMillis - maxFreshnessTs.getTime(); - metricsFacade.recordFreshness(freshnessMillis); - + metricsFacade.recordTableFreshness(tableName, freshnessMillis); } else { LOGGER.warn("Table {} returned null or zero max(freshness_ts). Skipping freshness calculation.", tableName); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java index 12ec3e4..cef2299 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java @@ -23,6 +23,7 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.util.DataTransform; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -54,7 +55,7 @@ public TableEventKafkaProvider(Properties kafkaProperties, String topic) throws this.kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, config.getGroupId() + "-" + topic); this.kafkaProperties.put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); this.kafkaProperties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 500); - this.tableName = extractTableName(topic); + this.tableName = DataTransform.extractTableName(topic); } @Override @@ -105,12 +106,6 @@ protected void processLoop() } } - private String extractTableName(String topic) - { - String[] parts = topic.split("\\."); - return parts[parts.length - 1]; - } - @Override RowChangeEvent convertToTargetRecord(Void record) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java index ab09b74..949131e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java @@ -22,6 +22,7 @@ import com.google.protobuf.InvalidProtocolBufferException; import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; @@ -39,12 +40,13 @@ public class TableEventStorageLoopProvider extends TableEventProvider { private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); private final boolean freshness_embed; - + private final boolean freshness_timestamp; protected TableEventStorageLoopProvider() { super(); - String sinkMonitorFreshnessLevel = PixelsSinkConfigFactory.getInstance().getSinkMonitorFreshnessLevel(); + PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + String sinkMonitorFreshnessLevel = config.getSinkMonitorFreshnessLevel(); if(sinkMonitorFreshnessLevel.equals("embed")) { freshness_embed = true; @@ -52,6 +54,7 @@ protected TableEventStorageLoopProvider() { freshness_embed = false; } + freshness_timestamp = config.isSinkMonitorFreshnessTimestamp(); } @Override @@ -65,11 +68,11 @@ RowChangeEvent convertToTargetRecord(T record) SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); SinkProto.RowRecord.Builder rowRecordBuilder = rowRecord.toBuilder(); - if(freshness_embed) + if(freshness_timestamp) { DataTransform.updateRecordTimestamp(rowRecordBuilder, System.currentTimeMillis() * 1000); - FreshnessClient.getInstance().addMonitoredTable(rowRecord.getSource().getTable()); } + SinkProto.TransactionInfo.Builder transactionBuilder = rowRecordBuilder.getTransactionBuilder(); String id = transactionBuilder.getId(); transactionBuilder.setId(id + "_" + loopId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java index 661f4c7..3c92c3a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -145,4 +145,10 @@ private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord rec SinkProto.RowRecord.Builder recordBuilder = record.toBuilder(); return updateRecordTimestamp(recordBuilder, timestampColumn); } + + public static String extractTableName(String topic) + { + String[] parts = topic.split("\\."); + return parts[parts.length - 1]; + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index 7f2b0fb..a3ea787 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -64,6 +64,7 @@ public class MetricsFacade private final Summary retinaServiceLatency; private final Summary writerLatency; private final Summary totalLatency; + private final Summary tableFreshness; private final Histogram transactionRowCountHistogram; private final Histogram primaryKeyUpdateDistribution; @@ -96,8 +97,6 @@ public class MetricsFacade private MetricsFacade(boolean enabled) { this.enabled = enabled; - if (enabled) - { this.debeziumEventCounter = Counter.build() .name("debezium_event_total") .help("Debezium Event Total") @@ -179,6 +178,7 @@ private MetricsFacade(boolean enabled) this.writerLatency = Summary.build() .name("write_latency_seconds") .help("Write latency") + .labelNames("table") .quantile(0.5, 0.05) .quantile(0.75, 0.01) .quantile(0.95, 0.005) @@ -194,6 +194,16 @@ private MetricsFacade(boolean enabled) .quantile(0.95, 0.005) .quantile(0.99, 0.001) .register(); + + this.tableFreshness = Summary.build() + .name("data_freshness_latency_ms") + .help("Data freshness latency in milliseconds per table") + .labelNames("table") + .quantile(0.5, 0.01) + .quantile(0.9, 0.01) + .quantile(0.99, 0.001) + .register(); + this.transactionRowCountHistogram = Histogram.build() .name("transaction_row_count_histogram") .help("Distribution of row counts within a single transaction") @@ -207,27 +217,6 @@ private MetricsFacade(boolean enabled) .register(); this.freshness = new SynchronizedDescriptiveStatistics(); this.rowChangeSpeed = new SynchronizedDescriptiveStatistics(); - } else - { - this.debeziumEventCounter = null; - this.rowEventCounter = null; - this.rowChangeCounter = null; - this.serdRowRecordCounter = null; - this.serdTxRecordCounter = null; - this.transactionCounter = null; - this.processingLatency = null; - this.tableChangeCounter = null; - this.rawDataThroughputCounter = null; - this.transServiceLatency = null; - this.indexServiceLatency = null; - this.retinaServiceLatency = null; - this.writerLatency = null; - this.totalLatency = null; - this.freshness = null; - this.rowChangeSpeed = null; - this.transactionRowCountHistogram = null; - this.primaryKeyUpdateDistribution = null; - } freshnessReportInterval = config.getFreshnessReportInterval(); freshnessReportPath = config.getMonitorFreshnessReportFile(); @@ -376,9 +365,9 @@ public Summary.Timer startRetinaLatencyTimer() return enabled ? retinaServiceLatency.startTimer() : null; } - public Summary.Timer startWriteLatencyTimer() + public Summary.Timer startWriteLatencyTimer(String tableName) { - return enabled ? writerLatency.startTimer() : null; + return enabled ? writerLatency.labels(tableName).startTimer() : null; } public void addRawData(double data) @@ -418,6 +407,17 @@ public int getTransactionEvent() return (int) transactionCounter.get(); } + public void recordTableFreshness(String table, double freshnessMill) + { + if(!enabled) + { + return; + } + + tableFreshness.labels(table).observe(freshnessMill); + recordFreshness(freshnessMill); + } + public void recordFreshness(double freshnessMill) { if(enabled && freshness != null) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index 728b9c2..bc3cc4d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -87,12 +87,14 @@ public SinkContext(String sourceTxId) { this.sourceTxId = sourceTxId; this.pixelsTransCtx = null; + setCurrStartTime(); } public SinkContext(String sourceTxId, TransContext pixelsTransCtx) { this.sourceTxId = sourceTxId; this.pixelsTransCtx = pixelsTransCtx; + setCurrStartTime(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 39d52e3..2bdd08e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -28,7 +28,9 @@ import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.freshness.FreshnessClient; import io.pixelsdb.pixels.sink.util.BlockingBoundedMap; +import io.pixelsdb.pixels.sink.util.DataTransform; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +51,7 @@ public class SinkContextManager private final TransactionProxy transactionProxy = TransactionProxy.Instance(); private final TableWriterProxy tableWriterProxy; private final CommitMethod commitMethod; - + private final String freshnessLevel; private SinkContextManager() { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); @@ -61,6 +63,7 @@ private SinkContextManager() { this.commitMethod = CommitMethod.Async; } + this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); } public static SinkContextManager getInstance() @@ -163,7 +166,6 @@ void processTxCommit(SinkProto.TransactionMetadata txEnd) throw new RuntimeException("Sink Context is null"); } - ctx.setStartTime(System.currentTimeMillis()); try { ctx.tableCounterLock.lock(); @@ -198,7 +200,14 @@ void endTransaction(SinkContext ctx) transactionProxy.commitTransAsync(ctx); } } - + if(freshnessLevel.equals("embed")) + { + for(String table: ctx.getTableCounters().keySet()) + { + String tableName = DataTransform.extractTableName(table); + FreshnessClient.getInstance().addMonitoredTable(tableName); + } + } } else { LOGGER.info("Abort transaction: {}", txId); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java index d21f875..4e6f78d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java @@ -26,6 +26,7 @@ import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.freshness.FreshnessClient; import io.pixelsdb.pixels.sink.util.DataTransform; +import io.prometheus.client.Summary; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,11 +96,18 @@ public void flush() { tableUpdateData.add(tableUpdateDataItem.build()); } + + final Summary.Timer startWriteLatencyTimer = metricsFacade.startWriteLatencyTimer(tableName); CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); updateRecordResponseCompletableFuture.thenAccept( resp -> { + if(freshness_embed) + { + FreshnessClient.getInstance().addMonitoredTable(tableName); + } + if (resp.getHeader().getErrorCode() != 0) { transactionProxy.rollbackTrans(pixelsTransContext); @@ -112,6 +120,10 @@ public void flush() metricsFacade.recordFreshness(txEndTime - txStartTime); } transactionProxy.commitTrans(pixelsTransContext); + if(startWriteLatencyTimer != null) + { + startWriteLatencyTimer.observeDuration(); + } } } ); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 920706d..1c0e904 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -68,7 +68,7 @@ public abstract class TableWriter protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); protected TransactionMode transactionMode; private final AtomicInteger counter = new AtomicInteger(); - + protected final boolean freshness_embed; protected TableWriter(String tableName, int bucketId) { this.config = PixelsSinkConfigFactory.getInstance(); @@ -79,7 +79,14 @@ protected TableWriter(String tableName, int bucketId) this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); this.delegate = new RetinaServiceProxy(bucketId); this.transactionMode = config.getTransactionMode(); - + String sinkMonitorFreshnessLevel = config.getSinkMonitorFreshnessLevel(); + if(sinkMonitorFreshnessLevel.equals("embed")) + { + freshness_embed = true; + } else + { + freshness_embed = false; + } if(this.config.isMonitorReportEnabled()) { long interval = this.config.getMonitorReportInterval(); @@ -111,7 +118,7 @@ private Runnable writerInfoTask(String tableName) { firstTx = firstEvent.getTransaction().getId(); int count = counter.get(); - getLOGGER().debug("{} Writer {}: Tx Now is {}. Buffer Len is {}. Total Count {}", reportId.incrementAndGet(), tableName, firstTx, len, count); + getLOGGER().info("{} Writer {}: Tx Now is {}. Buffer Len is {}. Total Count {}", reportId.incrementAndGet(), tableName, firstTx, len, count); } }; return monitorTask; From 53a74a60256d4e48bad56345ea0c27238ad3ee68 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Tue, 16 Dec 2025 08:02:00 +0000 Subject: [PATCH 45/53] Enhance Freshness & TableWriter --- conf/pixels-sink.aws.properties | 3 + perf_freshness.py | 3 +- .../pixels/sink/config/ConfigLoader.java | 14 +++ .../pixels/sink/config/PixelsSinkConfig.java | 10 ++ .../sink/freshness/FreshnessClient.java | 62 +++++++--- .../writer/retina/TableCrossTxWriter.java | 20 +--- .../retina/TableSingleRecordWriter.java | 23 +--- .../writer/retina/TableSingleTxWriter.java | 2 +- .../sink/writer/retina/TableWriter.java | 107 +++++++++++------- 9 files changed, 146 insertions(+), 98 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index fd80e14..8be7028 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -5,6 +5,7 @@ sink.datasource.rate.limit=20000 # Sink Config: retina | csv | proto | flink | none sink.mode=retina sink.retina.client=8 +sink.retina.log.queue=false ## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature sink.trans.mode=batch sink.monitor.report.enable=true @@ -18,6 +19,8 @@ trino.parallel=5 # row or txn or embed sink.monitor.freshness.level=embed sink.monitor.freshness.embed.warmup=10 +sink.monitor.freshness.embed.static=true +sink.monitor.freshness.embed.tablelist=savingaccount,transfer,loanapps,loantrans sink.monitor.freshness.verbose=true sink.monitor.freshness.timestamp=true sink.storage.loop=true diff --git a/perf_freshness.py b/perf_freshness.py index 51b0904..1b642ae 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -16,7 +16,8 @@ csv_files = { "Query Transaction": "tmp/i7i_2k_dec_freshness.csv", "Query Record": "tmp/i7i_2k_record_dec_freshness.csv", - "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv" + "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv", + "Query Selected Table, Trans Mode": "tmp/i7i_2k_batchtest_dec_freshness_2.csv" } MAX_SECONDS = 500 # 截取前多少秒的数据 SKIP_SECONDS = 20 # 跳过前多少秒的数据(可调) diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java index b3795cb..c53465d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java @@ -26,6 +26,7 @@ import io.pixelsdb.pixels.sink.writer.retina.TransactionMode; import java.lang.reflect.Field; +import java.util.List; import java.util.Properties; public class ConfigLoader @@ -92,6 +93,19 @@ private static Object convert(String value, Class type) } else if (type.equals(RetinaServiceProxy.RetinaWriteMode.class)) { return RetinaServiceProxy.RetinaWriteMode.fromValue(value); + } else if (type.equals(List.class)) + { + // Handle List type: split the string by comma (",") + // and return a List. Trimming each element is recommended. + if (value == null || value.isEmpty()) { + return java.util.Collections.emptyList(); + } + + // Use Arrays.asList(String.split(",")) to handle the splitting, + // then stream to trim whitespace from each element. + return java.util.Arrays.stream(value.split(",")) + .map(String::trim) + .collect(java.util.stream.Collectors.toList()); } return value; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 52c9f0b..9eab676 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import java.io.IOException; +import java.util.List; @Getter public class PixelsSinkConfig @@ -47,6 +48,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.retina.client", defaultValue = "1") private int retinaClientNum; + @ConfigKey(value = "sink.retina.log.queue", defaultValue = "true") + private boolean retinaLogQueueEnabled; + @ConfigKey(value = "sink.trans.mode", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_MODE) private TransactionMode transactionMode; @@ -182,6 +186,12 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.freshness.embed.warmup", defaultValue = "10") private Integer sinkMonitorFreshnessEmbedWarmupSeconds; + @ConfigKey(value = "sink.monitor.freshness.embed.static", defaultValue = "false") + private boolean sinkMonitorFreshnessEmbedStatic; + + @ConfigKey(value = "sink.monitor.freshness.embed.tablelist", defaultValue = "false") + private List sinkMonitorFreshnessEmbedTableList; + @ConfigKey(value = "sink.monitor.freshness.verbose", defaultValue = "false") private boolean sinkMonitorFreshnessVerbose; diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index cc4add5..4132dfb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -53,13 +53,13 @@ public class FreshnessClient private final ScheduledExecutorService scheduler; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final int warmUpSeconds; - + private final PixelsSinkConfig config; private FreshnessClient() { // Initializes the set with thread safety wrapper this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); - PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); + this.config = PixelsSinkConfigFactory.getInstance(); this.trinoUser = config.getTrinoUser(); this.trinoJdbcUrl = config.getTrinoUrl(); this.trinoPassword = config.getTrinoPassword(); @@ -239,22 +239,12 @@ void queryAndCalculateFreshness() try { - // Take a snapshot of the tables to monitor for this cycle. - // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. - Set tablesSnapshot = new HashSet<>(monitoredTables); - if (tablesSnapshot.isEmpty()) + tableName = getRandomTable(); + if(tableName == null) { - LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); return; } - monitoredTables.clear(); - List tableList = new ArrayList<>(tablesSnapshot); - - Random random = new Random(); - int randomIndex = random.nextInt(tableList.size()); - - tableName = tableList.get(randomIndex); - + LOGGER.debug("Randomly selected table for this cycle: {}", tableName); conn = createNewConnection(); // Timestamp when the query is sent (t_send) @@ -306,4 +296,46 @@ void queryAndCalculateFreshness() } } + + + private String getRandomTable() + { + List tableList; + if(config.isSinkMonitorFreshnessEmbedStatic()) + { + tableList = getStaticList(); + } else + { + tableList = getDynamicList(); + } + + if(tableList == null || tableList.isEmpty()) + { + return null; + } + + Random random = new Random(); + int randomIndex = random.nextInt(tableList.size()); + + return tableList.get(randomIndex); + } + + private List getDynamicList() + { + // Take a snapshot of the tables to monitor for this cycle. + // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. + Set tablesSnapshot = new HashSet<>(monitoredTables); + if (tablesSnapshot.isEmpty()) + { + LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); + return null; + } + monitoredTables.clear(); + List tableList = new ArrayList<>(tablesSnapshot); + return tableList; + } + private List getStaticList() + { + return config.getSinkMonitorFreshnessEmbedTableList(); + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 0d92fa9..a1da79a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -24,8 +24,6 @@ import io.pixelsdb.pixels.retina.RetinaProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.freshness.FreshnessClient; -import io.pixelsdb.pixels.sink.util.DataTransform; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,24 +59,8 @@ public TableCrossTxWriter(String t, int bucketId) /** * Flush any buffered events for the current transaction. */ - public void flush() + public void flush(List batch) { - List batch; - bufferLock.lock(); - try - { - if (buffer.isEmpty()) - { - return; - } - // Swap buffers quickly under lock - batch = buffer; - buffer = new LinkedList<>(); - } finally - { - bufferLock.unlock(); - } - writeLock.lock(); try { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java index 4e6f78d..ffddec9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java @@ -25,7 +25,6 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.freshness.FreshnessClient; -import io.pixelsdb.pixels.sink.util.DataTransform; import io.prometheus.client.Summary; import lombok.Getter; import org.slf4j.Logger; @@ -35,7 +34,6 @@ import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; public class TableSingleRecordWriter extends TableCrossTxWriter { @@ -52,26 +50,9 @@ public TableSingleRecordWriter(String t, int bucketId) /** * Flush any buffered events for the current transaction. */ - public void flush() + public void flush(List batch) { - List batch; - bufferLock.lock(); - try - { - if (buffer.isEmpty()) - { - return; - } - // Swap buffers quickly under lock - batch = buffer; - buffer = new LinkedList<>(); - } finally - { - bufferLock.unlock(); - } - - TransContext pixelsTransContext = transactionProxy.getNewTransContext(); - + TransContext pixelsTransContext = transactionProxy.getNewTransContext(tableName); writeLock.lock(); try { diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java index b757845..8afd8ce 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java @@ -44,7 +44,7 @@ public TableSingleTxWriter(String tableName, int bucketId) /** * Flush any buffered events for the current transaction. */ - public void flush() + public void flush(List batchToFlush) { List batch; String txId; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 1c0e904..01ee192 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -35,9 +35,9 @@ import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; /** @@ -50,9 +50,13 @@ public abstract class TableWriter { protected final RetinaServiceProxy delegate; // physical writer - protected final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); - protected final ScheduledExecutorService logScheduler = Executors.newScheduledThreadPool(1); + + private final ScheduledExecutorService flushExecutor = Executors.newSingleThreadScheduledExecutor(); + private final ScheduledExecutorService logScheduler = Executors.newScheduledThreadPool(1); protected final ReentrantLock bufferLock = new ReentrantLock(); + protected final Condition flushCondition = bufferLock.newCondition(); + protected final Thread flusherThread; + protected volatile boolean running = true; protected final String tableName; protected final long flushInterval; protected final FlushRateLimiter flushRateLimiter; @@ -62,7 +66,6 @@ public abstract class TableWriter protected List buffer = new LinkedList<>(); protected volatile String currentTxId = null; protected String txId = null; - protected ScheduledFuture flushTask = null; protected String fullTableName; protected PixelsSinkConfig config; protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); @@ -87,7 +90,7 @@ protected TableWriter(String tableName, int bucketId) { freshness_embed = false; } - if(this.config.isMonitorReportEnabled()) + if(this.config.isMonitorReportEnabled() && this.config.isRetinaLogQueueEnabled()) { long interval = this.config.getMonitorReportInterval(); Runnable monitorTask = writerInfoTask(tableName); @@ -98,8 +101,52 @@ protected TableWriter(String tableName, int bucketId) TimeUnit.MILLISECONDS ); } + this.flusherThread = new Thread(new FlusherRunnable(), "Pixels-Flusher-" + tableName); + this.flusherThread.start(); } + private class FlusherRunnable implements Runnable { + @Override + public void run() { + while (running) { + bufferLock.lock(); + try { + if (!needFlush()) + { + try + { + // Conditional wait: will wait until signaled by write() or timeout + flushCondition.await(flushInterval, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // Exit loop if interrupted during shutdown + running = false; + Thread.currentThread().interrupt(); + return; + } + } + + List batchToFlush = buffer; + buffer = new LinkedList<>(); + bufferLock.unlock(); + submitFlushTask(batchToFlush); + bufferLock.lock(); + } finally + { + bufferLock.unlock(); + } + } + } + } + private void submitFlushTask(List batch) + { + if(batch == null || batch.isEmpty()) + { + return; + } + flushExecutor.submit(() -> { + flush(batch); + }); + } private Runnable writerInfoTask(String tableName) { final AtomicInteger reportId = new AtomicInteger(); @@ -172,16 +219,6 @@ public boolean write(RowChangeEvent event, SinkContext ctx) { txId = ctx.getSourceTxId(); } - // If this is a new transaction, flush the old one - if (needFlush()) - { - if (flushTask != null) - { - flushTask.cancel(false); - } - flush(); - - } currentTxId = txId; if (fullTableName == null) { @@ -190,31 +227,10 @@ public boolean write(RowChangeEvent event, SinkContext ctx) counter.incrementAndGet(); buffer.add(event); - // Reset scheduled flush: cancel old one and reschedule - if (flushTask != null && !flushTask.isDone()) + if (needFlush()) { - flushTask.cancel(false); + flushCondition.signalAll(); } - flushTask = scheduler.schedule(() -> - { - try - { - bufferLock.lock(); - try - { - if (transactionMode.equals(TransactionMode.RECORD) || txId.equals(currentTxId)) - { - flush(); - } - } finally - { - bufferLock.unlock(); - } - } catch (Exception e) - { - getLOGGER().error("Scheduled flush failed for table {}", tableName, e); - } - }, flushInterval, TimeUnit.MILLISECONDS); } finally { bufferLock.unlock(); @@ -227,17 +243,26 @@ public boolean write(RowChangeEvent event, SinkContext ctx) } } - public abstract void flush(); + public abstract void flush(List batchToFlush); protected abstract boolean needFlush(); public void close() { - scheduler.shutdown(); + this.running = false; + if (this.flusherThread != null) + { + this.flusherThread.interrupt(); + } logScheduler.shutdown(); try { - scheduler.awaitTermination(5, TimeUnit.SECONDS); + logScheduler.awaitTermination(5, TimeUnit.SECONDS); + flushExecutor.awaitTermination(5, TimeUnit.SECONDS); + if (this.flusherThread != null) + { + this.flusherThread.join(5000); + } delegate.close(); } catch (InterruptedException ignored) { From f82964ae0f3cd375a7d4a7b5704798e5f22140f6 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 17 Dec 2025 07:38:01 +0000 Subject: [PATCH 46/53] Freshness Snapshot Query --- .gitignore | 1 + conf/pixels-sink.aws.properties | 12 +++-- .../pixels/sink/config/PixelsSinkConfig.java | 3 ++ .../sink/freshness/FreshnessClient.java | 53 +++++++++++++++++-- .../sink/freshness/TestFreshnessClient.java | 11 ++++ 5 files changed, 71 insertions(+), 9 deletions(-) diff --git a/.gitignore b/.gitignore index db6976d..5b77bdc 100644 --- a/.gitignore +++ b/.gitignore @@ -54,3 +54,4 @@ tmp/ !requirements.txt freshness*.png rate*.png +resulti7i/ diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 8be7028..6abd538 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,7 +1,7 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=20000 +sink.datasource.rate.limit=10000 # Sink Config: retina | csv | proto | flink | none sink.mode=retina sink.retina.client=8 @@ -9,18 +9,20 @@ sink.retina.log.queue=false ## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature sink.trans.mode=batch sink.monitor.report.enable=true -sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/i7i_2k_batchtest_dec_rate.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/i7i_2k_batchtest_dec_freshness_2.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/10k_rate.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/10k_freshness.csv # trino for freshness query trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x +# trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x trino.user=pixels trino.password=password -trino.parallel=5 +trino.parallel=4 # row or txn or embed sink.monitor.freshness.level=embed sink.monitor.freshness.embed.warmup=10 sink.monitor.freshness.embed.static=true -sink.monitor.freshness.embed.tablelist=savingaccount,transfer,loanapps,loantrans +sink.monitor.freshness.embed.snapshot=true +sink.monitor.freshness.embed.tablelist=loanapps,loantrans sink.monitor.freshness.verbose=true sink.monitor.freshness.timestamp=true sink.storage.loop=true diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index 9eab676..e86ef83 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -189,6 +189,9 @@ public class PixelsSinkConfig @ConfigKey(value = "sink.monitor.freshness.embed.static", defaultValue = "false") private boolean sinkMonitorFreshnessEmbedStatic; + @ConfigKey(value = "sink.monitor.freshness.embed.snapshot", defaultValue = "false") + private boolean sinkMonitorFreshnessEmbedSnapshot; + @ConfigKey(value = "sink.monitor.freshness.embed.tablelist", defaultValue = "false") private List sinkMonitorFreshnessEmbedTableList; diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 4132dfb..3dac8ad 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -20,6 +20,9 @@ package io.pixelsdb.pixels.sink.freshness; +import io.pixelsdb.pixels.common.exception.TransException; +import io.pixelsdb.pixels.common.transaction.TransContext; +import io.pixelsdb.pixels.common.transaction.TransService; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.util.DateUtil; @@ -109,7 +112,8 @@ public static FreshnessClient getInstance() return instance; } - private Connection createNewConnection() throws SQLException + @Deprecated + protected Connection createNewConnection() throws SQLException { try { @@ -118,9 +122,32 @@ private Connection createNewConnection() throws SQLException { throw new SQLException(e); } + + Properties properties = new Properties(); + + return DriverManager.getConnection(trinoJdbcUrl, trinoUser, null); } + protected Connection createNewConnection(long queryTimestamp) throws SQLException + { + try + { + Class.forName("io.trino.jdbc.TrinoDriver"); + } catch (ClassNotFoundException e) + { + throw new SQLException(e); + } + + Properties properties = new Properties(); + properties.setProperty("user", trinoUser); + String catalogName = "pixels"; + String sessionPropValue = String.format("%s.query_snapshot_timestamp:%d", catalogName, queryTimestamp); + + properties.setProperty("sessionProperties", sessionPropValue); + return DriverManager.getConnection(trinoJdbcUrl, properties); + } + private void closeConnection(Connection conn) { if (conn != null) @@ -233,7 +260,7 @@ private void submitQueryTask() void queryAndCalculateFreshness() { Connection conn = null; - + TransContext transContext = null; String tableName; try @@ -244,11 +271,19 @@ void queryAndCalculateFreshness() { return; } - + LOGGER.debug("Randomly selected table for this cycle: {}", tableName); - conn = createNewConnection(); // Timestamp when the query is sent (t_send) long tSendMillis = System.currentTimeMillis(); + if(config.isSinkMonitorFreshnessEmbedSnapshot()) + { + transContext = TransService.Instance().beginTrans(true); + conn = createNewConnection(transContext.getTimestamp()); + } else + { + conn = createNewConnection(); + } + String tSendMillisStr = DateUtil.convertDateToString(new Date(tSendMillis)); // Query to find the latest timestamp in the table // Assumes 'freshness_ts' is a long-type epoch timestamp (milliseconds) @@ -291,6 +326,16 @@ void queryAndCalculateFreshness() LOGGER.error("Error selecting a random table from the monitor list.", e); } finally { + if(config.isSinkMonitorFreshnessEmbedSnapshot() && transContext != null) + { + try + { + TransService.Instance().commitTrans(transContext.getTransId(), true); + } catch (TransException e) + { + throw new RuntimeException(e); + } + } closeConnection(conn); queryPermits.release(); } diff --git a/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java b/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java index 4123b29..6535ad8 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java +++ b/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java @@ -49,4 +49,15 @@ public void testFreshnessCalculationSuccess() throws Exception { freshnessClient.start(); while(true){} } + + @Test + public void testSnapshotTs() throws SQLException + { + FreshnessClient freshnessClient = FreshnessClient.getInstance(); + Connection connection = freshnessClient.createNewConnection(123456L); + String query = String.format("SELECT max(freshness_ts) FROM customer"); + Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(query); + resultSet.next(); + } } \ No newline at end of file From 83dacac2a639cdd52ecf2cd02c07bf8eb34dc7d2 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 18 Dec 2025 06:20:13 +0000 Subject: [PATCH 47/53] Improve FreshnessClient --- conf/pixels-sink.aws.properties | 8 ++--- perf_freshness.py | 31 ++++++++++--------- .../sink/freshness/FreshnessClient.java | 19 ++++++++++-- 3 files changed, 37 insertions(+), 21 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 6abd538..2298d02 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,7 +1,7 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=10000 +sink.datasource.rate.limit=50000 # Sink Config: retina | csv | proto | flink | none sink.mode=retina sink.retina.client=8 @@ -9,8 +9,8 @@ sink.retina.log.queue=false ## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature sink.trans.mode=batch sink.monitor.report.enable=true -sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/10k_rate.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/10k_freshness.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/50k_rate.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/50k_freshness.csv # trino for freshness query trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x # trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x @@ -20,7 +20,7 @@ trino.parallel=4 # row or txn or embed sink.monitor.freshness.level=embed sink.monitor.freshness.embed.warmup=10 -sink.monitor.freshness.embed.static=true +sink.monitor.freshness.embed.static=false sink.monitor.freshness.embed.snapshot=true sink.monitor.freshness.embed.tablelist=loanapps,loantrans sink.monitor.freshness.verbose=true diff --git a/perf_freshness.py b/perf_freshness.py index 1b642ae..f081494 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -5,24 +5,25 @@ ########################################## # 配置 CSV 文件 和 标签 ########################################## -# csv_files = { -# "10k": "tmp/freshness10k_5.csv", -# "20k": "tmp/freshness20k_5.csv", -# # "30k": "tmp/freshness30k_2.csv", -# "40k": "tmp/freshness40k_5.csv", -# "50k": "tmp/freshness50k_5.csv", -# "60k": "tmp/freshness60k_5.csv" -# } csv_files = { - "Query Transaction": "tmp/i7i_2k_dec_freshness.csv", - "Query Record": "tmp/i7i_2k_record_dec_freshness.csv", - "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv", - "Query Selected Table, Trans Mode": "tmp/i7i_2k_batchtest_dec_freshness_2.csv" + "10k": "resulti7i/10k_freshness.csv", + "20k": "resulti7i/20k_freshness.csv", + "30k": "resulti7i/30k_freshness.csv", + "40k": "resulti7i/40k_freshness.csv", + # "40k": "tmp/freshness40k_5.csv", + # "50k": "tmp/freshness50k_5.csv", + # "60k": "tmp/freshness60k_5.csv" } -MAX_SECONDS = 500 # 截取前多少秒的数据 -SKIP_SECONDS = 20 # 跳过前多少秒的数据(可调) +# csv_files = { +# "Query Transaction": "tmp/i7i_2k_dec_freshness.csv", +# "Query Record": "tmp/i7i_2k_record_dec_freshness.csv", +# "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv", +# "Query Selected Table, Trans Mode": "tmp/i7i_2k_batchtest_dec_freshness_2.csv" +# } +MAX_SECONDS = 2000 # 截取前多少秒的数据 +SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) BIN_SECONDS = 10 # 平均窗口(秒) -MAX_FRESHNESS = 5000 +MAX_FRESHNESS = 5000 # 过滤初始warmup时的无用数据 ########################################## # 加载并处理数据 ########################################## diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 3dac8ad..30f06ce 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -370,15 +370,30 @@ private List getDynamicList() // Take a snapshot of the tables to monitor for this cycle. // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. Set tablesSnapshot = new HashSet<>(monitoredTables); + if (tablesSnapshot.isEmpty()) { LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); return null; } + monitoredTables.clear(); - List tableList = new ArrayList<>(tablesSnapshot); - return tableList; + + List staticList = getStaticList(); + + // If staticList is empty or null, return all tablesSnapshot + if (staticList == null || staticList.isEmpty()) + { + return new ArrayList<>(tablesSnapshot); + } + + // Return intersection of tablesSnapshot and staticList + Set staticSet = new HashSet<>(staticList); + tablesSnapshot.retainAll(staticSet); + + return new ArrayList<>(tablesSnapshot); } + private List getStaticList() { return config.getSinkMonitorFreshnessEmbedTableList(); From a56ad3c562425ad8f86058c883e2bbaf9f3eb1a9 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 18 Dec 2025 07:17:20 +0000 Subject: [PATCH 48/53] Feature: Flink & Retina Implement Abstract Bucket Writer --- conf/pixels-sink.flink.properties | 46 +++++++ .../sink/writer/AbstractBucketedWriter.java | 117 ++++++++++++++++++ .../sink/writer/PixelsSinkWriterFactory.java | 2 + .../sink/writer/{ => csv}/CsvWriter.java | 4 +- .../sink/writer/flink/FlinkPollingWriter.java | 106 ++++++++++------ .../flink/PixelsPollingServiceImpl.java | 29 ++++- .../sink/writer/{ => proto}/ProtoWriter.java | 3 +- .../{ => proto}/RotatingWriterManager.java | 2 +- .../writer/retina/RetinaBucketDispatcher.java | 42 +++++++ .../writer/retina/SinkContextManager.java | 65 +--------- .../pixels/sink/writer/TestProtoWriter.java | 1 + 11 files changed, 307 insertions(+), 110 deletions(-) create mode 100644 conf/pixels-sink.flink.properties create mode 100644 src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java rename src/main/java/io/pixelsdb/pixels/sink/writer/{ => csv}/CsvWriter.java (97%) rename src/main/java/io/pixelsdb/pixels/sink/writer/{ => proto}/ProtoWriter.java (99%) rename src/main/java/io/pixelsdb/pixels/sink/writer/{ => proto}/RotatingWriterManager.java (98%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java diff --git a/conf/pixels-sink.flink.properties b/conf/pixels-sink.flink.properties new file mode 100644 index 0000000..f0fb372 --- /dev/null +++ b/conf/pixels-sink.flink.properties @@ -0,0 +1,46 @@ +# engine | kafka | storage +sink.datasource=storage +# -1 means no limit, Only implement in retina sink mode yet +sink.datasource.rate.limit=50000 +# Sink Config: retina | csv | proto | flink | none +sink.mode=flink +## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature +sink.trans.mode=batch +sink.monitor.report.enable=true +sink.monitor.report.file=/home/ubuntu/pixels-sink/tmp/test.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/tmp/test_freshness.csv +# trino for freshness query +trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x +# trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x +trino.user=pixels +trino.password=password +trino.parallel=1 +# row or txn or embed +sink.monitor.freshness.level=row +sink.monitor.freshness.embed.warmup=10 +sink.monitor.freshness.embed.static=false +sink.monitor.freshness.embed.snapshot=true +sink.monitor.freshness.embed.tablelist=loanapps,loantrans +sink.monitor.freshness.verbose=false +sink.monitor.freshness.timestamp=true +sink.storage.loop=true + +sink.remote.host=localhost +sink.remote.port=29422 +sink.timeout.ms=5000 +sink.flush.interval.ms=50 +sink.flush.batch.size=10 +sink.max.retries=3 + +## Proto Config +sink.proto.dir=file:///home/ubuntu/disk1/hybench/ +sink.proto.data=hybench10_10 +sink.proto.maxRecords=100000 +## Flink Config +sink.flink.server.port=9091 + +# Sink Metrics +sink.monitor.enable=true +sink.monitor.port=9465 +sink.monitor.report.interval=10000 +sink.monitor.freshness.interval=1000 diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java new file mode 100644 index 0000000..1ad46b6 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java @@ -0,0 +1,117 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +package io.pixelsdb.pixels.sink.writer; + +import io.pixelsdb.pixels.sink.SinkProto; +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.exception.SinkException; + +public abstract class AbstractBucketedWriter +{ + public void writeRowChangeEvent(RowChangeEvent event, C context) throws SinkException + { + if (event == null) + { + return; + } + + event.initIndexKey(); + + switch (event.getOp()) + { + case UPDATE -> + { + if (!event.isPkChanged()) + { + emitBefore(event, context); + } + else + { + emitPkChangedUpdate(event, context); + } + } + + case DELETE -> emitBefore(event, context); + + case INSERT, SNAPSHOT -> emitAfter(event, context); + + case UNRECOGNIZED -> + { + return; + } + } + } + + /* ================= hook points ================= */ + + protected void emitBefore(RowChangeEvent event, C context) + { + int bucketId = event.getBeforeBucketFromIndex(); + emit(event, bucketId, context); + } + + protected void emitAfter(RowChangeEvent event, C context) + { + int bucketId = event.getAfterBucketFromIndex(); + emit(event, bucketId, context); + } + + protected void emitPkChangedUpdate(RowChangeEvent event, C context) throws SinkException + { + // DELETE (before) + RowChangeEvent deleteEvent = buildDeleteEvent(event); + emitBefore(deleteEvent, context); + + // INSERT (after) + RowChangeEvent insertEvent = buildInsertEvent(event); + emitAfter(insertEvent, context); + } + + protected abstract void emit(RowChangeEvent event, int bucketId, C context); + + /* ================= helpers ================= */ + + private RowChangeEvent buildDeleteEvent(RowChangeEvent event) throws SinkException + { + SinkProto.RowRecord.Builder builder = + event.getRowRecord().toBuilder() + .clearAfter() + .setOp(SinkProto.OperationType.DELETE); + + RowChangeEvent deleteEvent = + new RowChangeEvent(builder.build(), event.getSchema()); + deleteEvent.initIndexKey(); + return deleteEvent; + } + + private RowChangeEvent buildInsertEvent(RowChangeEvent event) throws SinkException + { + SinkProto.RowRecord.Builder builder = + event.getRowRecord().toBuilder() + .clearBefore() + .setOp(SinkProto.OperationType.INSERT); + + RowChangeEvent insertEvent = + new RowChangeEvent(builder.build(), event.getSchema()); + insertEvent.initIndexKey(); + return insertEvent; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java index 7e39555..20cef21 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java @@ -22,6 +22,8 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.writer.csv.CsvWriter; +import io.pixelsdb.pixels.sink.writer.proto.ProtoWriter; import io.pixelsdb.pixels.sink.writer.retina.RetinaWriter; import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java similarity index 97% rename from src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java index b577d2f..85edf57 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/CsvWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.writer; +package io.pixelsdb.pixels.sink.writer.csv; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.pixelsdb.pixels.sink.SinkProto; @@ -26,6 +26,8 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkDefaultConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.writer.PixelsSinkMode; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java index 10e82e5..2a20095 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java @@ -26,6 +26,7 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.writer.AbstractBucketedWriter; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,12 +47,12 @@ * This class is thread-safe and integrates FlushRateLimiter to control ingress traffic. * It also manages the lifecycle of the gRPC server. */ -public class FlinkPollingWriter implements PixelsSinkWriter { +public class FlinkPollingWriter extends AbstractBucketedWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(FlinkPollingWriter.class); - + record TableBucketKey(SchemaTableName table, int bucketId) {} // Core data structure: A thread-safe map from table name to a thread-safe blocking queue. - private final Map> tableQueues; + private final Map> tableQueues; // The gRPC server instance managed by this writer. private final PollingRpcServer pollingRpcServer; @@ -62,7 +63,6 @@ public class FlinkPollingWriter implements PixelsSinkWriter { */ public FlinkPollingWriter() { this.tableQueues = new ConcurrentHashMap<>(); - LOGGER.info("FlinkPollingWriter initialized with FlushRateLimiter."); // --- START: New logic to initialize and start the gRPC server --- try { @@ -101,31 +101,20 @@ public boolean writeRow(RowChangeEvent event) { return false; } - try { - // 2. Convert Flink's RowChangeEvent to the gRPC RowRecord Protobuf object - SinkProto.RowRecord rowRecord = event.getRowRecord(); - - // 3. Find the corresponding queue for the table name, creating a new one atomically if it doesn't exist. - BlockingQueue queue = tableQueues.computeIfAbsent( - event.getSchemaTableName(), - k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE) // Default to an unbounded queue + try + { + writeRowChangeEvent(event, null); + return true; + } + catch (Exception e) + { + LOGGER.error( + "Failed to process and write row for table: {}", + event.getFullTableName(), + e ); - - // 4. Put the converted record into the queue. - queue.put(rowRecord); - - LOGGER.debug("Enqueued a row for table '{}'. Queue size is now {}.", event.getFullTableName(), queue.size()); - - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); // Restore the interrupted status - LOGGER.error("Thread was interrupted while writing row for table: " + event.getFullTableName(), e); - return false; - } catch (Exception e) { - LOGGER.error("Failed to process and write row for table: " + event.getFullTableName(), e); return false; } - - return true; } /** @@ -140,33 +129,42 @@ public boolean writeRow(RowChangeEvent event) { * @return A list of RowRecords, which will be empty if no data is available before the timeout. * @throws InterruptedException if the thread is interrupted while waiting */ - public List pollRecords(SchemaTableName tableName, int batchSize, long timeout, TimeUnit unit) - throws InterruptedException { + public List pollRecords( + SchemaTableName tableName, + int bucketId, + int batchSize, + long timeout, + TimeUnit unit + ) throws InterruptedException + { List records = new ArrayList<>(batchSize); - BlockingQueue queue = tableQueues.get(tableName); + TableBucketKey key = new TableBucketKey(tableName, bucketId); - if (queue == null) { - // If the queue doesn't exist yet, wait for the specified timeout to simulate polling behavior. + BlockingQueue queue = tableQueues.get(key); + + if (queue == null) + { unit.sleep(timeout); return records; } - // Wait for the first record for up to the timeout period. - SinkProto.RowRecord firstRecord = queue.poll(timeout, unit); - if (firstRecord == null) { - // Timeout occurred, no records available. + SinkProto.RowRecord first = queue.poll(timeout, unit); + if (first == null) + { return records; } - // At least one record was found, add it to the list. - records.add(firstRecord); - // Drain any remaining records up to the batch size limit without blocking. + records.add(first); queue.drainTo(records, batchSize - 1); - LOGGER.info("Polled {} records for table '{}'.", records.size(), tableName); + LOGGER.info( + "Polled {} records for table {}, bucket {}", + records.size(), tableName, bucketId + ); return records; } + /** * This implementation does not involve transactions, so this method is a no-op. */ @@ -199,4 +197,34 @@ public void close() throws IOException tableQueues.clear(); LOGGER.info("FlinkPollingWriter closed."); } + + @Override + protected void emit(RowChangeEvent event, int bucketId, Void unused) + { + TableBucketKey key = + new TableBucketKey(event.getSchemaTableName(), bucketId); + + BlockingQueue queue = + tableQueues.computeIfAbsent( + key, + k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE) + ); + + try + { + queue.put(event.getRowRecord()); + LOGGER.debug( + "Enqueued row for table {}, bucket {}, queueSize={}", + event.getFullTableName(), bucketId, queue.size() + ); + } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new RuntimeException( + "Interrupted while enqueueing row for " + event.getFullTableName(), + e + ); + } + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java index 75e293f..0db65f3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java @@ -70,13 +70,30 @@ public PixelsPollingServiceImpl(FlinkPollingWriter writer) { public void pollEvents(SinkProto.PollRequest request, StreamObserver responseObserver) { SchemaTableName schemaTableName = new SchemaTableName(request.getSchemaName(), request.getTableName()); LOGGER.debug("Received poll request for table '{}'", schemaTableName); + List records = new ArrayList<>(pollBatchSize); + try { - List records = writer.pollRecords( - schemaTableName, - pollBatchSize, - pollTimeoutMs, - TimeUnit.MILLISECONDS - ); + for (int bucketId : request.getBucketsList()) + { + if (records.size() >= pollBatchSize) + { + break; + } + + List polled = + writer.pollRecords( + schemaTableName, + bucketId, + pollBatchSize - records.size(), + 0, + TimeUnit.MILLISECONDS + ); + + if (polled != null && !polled.isEmpty()) + { + records.addAll(polled); + } + } SinkProto.PollResponse.Builder responseBuilder = SinkProto.PollResponse.newBuilder(); if(records != null && !records.isEmpty()) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java similarity index 99% rename from src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java index 6fd534f..1fcf6e5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.writer; +package io.pixelsdb.pixels.sink.writer.proto; import io.pixelsdb.pixels.common.physical.PhysicalWriter; @@ -29,6 +29,7 @@ import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.util.TableCounters; +import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java similarity index 98% rename from src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java rename to src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java index 4dc802a..f4ecf5d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/RotatingWriterManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.writer; +package io.pixelsdb.pixels.sink.writer.proto; import io.pixelsdb.pixels.common.physical.PhysicalWriter; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java new file mode 100644 index 0000000..9dbb492 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java @@ -0,0 +1,42 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +package io.pixelsdb.pixels.sink.writer.retina; + +import io.pixelsdb.pixels.sink.event.RowChangeEvent; +import io.pixelsdb.pixels.sink.writer.AbstractBucketedWriter; + +public class RetinaBucketDispatcher extends AbstractBucketedWriter +{ + private final TableWriterProxy tableWriterProxy; + + public RetinaBucketDispatcher() + { + this.tableWriterProxy = TableWriterProxy.getInstance(); + } + + @Override + protected void emit(RowChangeEvent event, int bucketId, SinkContext ctx) + { + tableWriterProxy + .getTableWriter(event.getTable(), event.getTableId(), bucketId) + .write(event, ctx); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 2bdd08e..6666224 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -49,13 +49,12 @@ public class SinkContextManager private final BlockingBoundedMap activeTxContexts = new BlockingBoundedMap<>(100000); // private final ConcurrentMap activeTxContexts = new ConcurrentHashMap<>(10000); private final TransactionProxy transactionProxy = TransactionProxy.Instance(); - private final TableWriterProxy tableWriterProxy; private final CommitMethod commitMethod; private final String freshnessLevel; + private final RetinaBucketDispatcher retinaBucketDispatcher; private SinkContextManager() { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - this.tableWriterProxy = TableWriterProxy.getInstance(); if (config.getCommitMethod().equals("sync")) { this.commitMethod = CommitMethod.Sync; @@ -64,6 +63,7 @@ private SinkContextManager() this.commitMethod = CommitMethod.Async; } this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); + this.retinaBucketDispatcher = new RetinaBucketDispatcher(); } public static SinkContextManager getInstance() @@ -244,66 +244,7 @@ protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws { event.setTimeStamp(ctx.getTimestamp()); } - event.initIndexKey(); - switch (event.getOp()) - { - case UPDATE -> - { - if (!event.isPkChanged()) - { - writeBeforeEvent(ctx, event); - } else - { - TypeDescription typeDescription = event.getSchema(); - if(ctx != null) - { - ctx.updateCounter(event.getFullTableName(), -1L); - } - SinkProto.RowRecord.Builder deleteBuilder = event.getRowRecord().toBuilder() - .clearAfter().setOp(SinkProto.OperationType.DELETE); - RowChangeEvent deleteEvent = new RowChangeEvent(deleteBuilder.build(), typeDescription); - deleteEvent.initIndexKey(); - writeBeforeEvent(ctx, deleteEvent); - - SinkProto.RowRecord.Builder insertBuilder = event.getRowRecord().toBuilder() - .clearBefore().setOp(SinkProto.OperationType.INSERT); - RowChangeEvent insertEvent = new RowChangeEvent(insertBuilder.build(), typeDescription); - insertEvent.initIndexKey(); - writeAfterEvent(ctx, deleteEvent); - } - } - case DELETE -> - { - writeBeforeEvent(ctx, event); - } - case INSERT, SNAPSHOT -> - { - writeAfterEvent(ctx, event); - } - case UNRECOGNIZED -> - { - return; - } - } - } - - private boolean writeBeforeEvent(SinkContext ctx, RowChangeEvent event) - { - int beforeBucketFromIndex = event.getBeforeBucketFromIndex(); - return writeBucketEvent(ctx, event, beforeBucketFromIndex); - } - - private boolean writeAfterEvent(SinkContext ctx, RowChangeEvent event) - { - int afterBucketFromIndex = event.getAfterBucketFromIndex(); - return writeBucketEvent(ctx, event, afterBucketFromIndex); - } - - private boolean writeBucketEvent(SinkContext ctx, RowChangeEvent event, int bucketId) - { - String table = event.getTable(); - long tableId = event.getTableId(); - return tableWriterProxy.getTableWriter(table, tableId, bucketId).write(event, ctx); + retinaBucketDispatcher.writeRowChangeEvent(event, ctx); } protected SinkContext getSinkContext(String txId) diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java index 5fa8247..0c932a3 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TestProtoWriter.java @@ -23,6 +23,7 @@ import io.pixelsdb.pixels.common.physical.*; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.writer.proto.ProtoWriter; import io.pixelsdb.pixels.storage.localfs.PhysicalLocalReader; import lombok.SneakyThrows; import org.junit.jupiter.api.BeforeAll; From fb29502d6d5964a2174e7a6ba0f3727073c39346 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 18 Dec 2025 07:58:32 +0000 Subject: [PATCH 49/53] comment --- .../io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java | 2 +- .../pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index 4d1212e..ad3a03c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -71,7 +71,7 @@ public TableMetadata getMetadata(String schema, String table) throws SinkExcepti SchemaTableName key = new SchemaTableName(schema, table); if (!registry.containsKey(key)) { - logger.info("Registry doesn't contain {}", key); + logger.debug("Registry doesn't contain {}", key); TableMetadata metadata = loadTableMetadata(schema, table); registry.put(key, metadata); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java index 2a20095..4a650b1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java @@ -123,6 +123,7 @@ public boolean writeRow(RowChangeEvent event) { * batchSize acts as an upper limit on the number of records pulled to prevent oversized RPC responses. * * @param tableName The name of the table to pull data from + * @param bucketId * @param batchSize The maximum number of records to pull * @param timeout The maximum time to wait for data * @param unit The time unit for the timeout From 5bd386098fa77fc5814328f6da7db09f83affbe9 Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Thu, 18 Dec 2025 13:16:40 +0000 Subject: [PATCH 50/53] update perf scripts --- conf/pixels-sink.aws.properties | 8 +-- perf_freshness.py | 45 +++++++++----- perf_rate.py | 106 +++++++++++++++++--------------- 3 files changed, 88 insertions(+), 71 deletions(-) diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index 2298d02..e18dbc2 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,7 +1,7 @@ # engine | kafka | storage sink.datasource=storage # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=50000 +sink.datasource.rate.limit=100000 # Sink Config: retina | csv | proto | flink | none sink.mode=retina sink.retina.client=8 @@ -9,14 +9,14 @@ sink.retina.log.queue=false ## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature sink.trans.mode=batch sink.monitor.report.enable=true -sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/50k_rate.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/50k_freshness.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/100k_rate_2.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/100k_freshness_2.csv # trino for freshness query trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x # trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x trino.user=pixels trino.password=password -trino.parallel=4 +trino.parallel=8 # row or txn or embed sink.monitor.freshness.level=embed sink.monitor.freshness.embed.warmup=10 diff --git a/perf_freshness.py b/perf_freshness.py index f081494..b8bcbdd 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -6,13 +6,16 @@ # 配置 CSV 文件 和 标签 ########################################## csv_files = { - "10k": "resulti7i/10k_freshness.csv", - "20k": "resulti7i/20k_freshness.csv", - "30k": "resulti7i/30k_freshness.csv", - "40k": "resulti7i/40k_freshness.csv", - # "40k": "tmp/freshness40k_5.csv", - # "50k": "tmp/freshness50k_5.csv", - # "60k": "tmp/freshness60k_5.csv" + # "10k_2": "resulti7i/10k_freshness.csv", + "10k": "resulti7i/10k_freshness_2.csv", + # "20k": "resulti7i/20k_freshness.csv", + "20k": "resulti7i/20k_freshness_2.csv", + "30k": "resulti7i/30k_freshness_2.csv", + "40k": "resulti7i/40k_freshness_2.csv", + "50k": "resulti7i/50k_freshness.csv", + "60k": "resulti7i/60k_freshness_2.csv", + "80k": "resulti7i/80k_freshness_2.csv", + "100k": "resulti7i/100k_freshness_2.csv", } # csv_files = { # "Query Transaction": "tmp/i7i_2k_dec_freshness.csv", @@ -20,10 +23,10 @@ # "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv", # "Query Selected Table, Trans Mode": "tmp/i7i_2k_batchtest_dec_freshness_2.csv" # } -MAX_SECONDS = 2000 # 截取前多少秒的数据 +MAX_SECONDS = 1800 # 截取前多少秒的数据 SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) BIN_SECONDS = 10 # 平均窗口(秒) -MAX_FRESHNESS = 5000 # 过滤初始warmup时的无用数据 +MAX_FRESHNESS = 500000 # 过滤初始warmup时的无用数据 ########################################## # 加载并处理数据 ########################################## @@ -81,24 +84,32 @@ ########################################## -# 图 2:CDF(同样使用平均窗口后的数据) +# 图 2:翻转轴后的 CDF(X轴 0-1,步长 0.1) ########################################## plt.figure(figsize=(10, 5)) for label, df in data.items(): vals = np.sort(df["freshness"].dropna()) - y = np.linspace(0, 1, len(vals)) - plt.plot(vals, y, label=label) + prob = np.linspace(0, 1, len(vals)) + + # x轴为概率 [0, 1],y轴为数值 + plt.plot(prob, vals, label=label) -plt.xscale("log") -plt.xlabel(f"Freshness (ms, {BIN_SECONDS}s average)") -plt.ylabel("CDF") +# 设置 X 轴刻度:从 0 到 1.1(不包含1.1),步长 0.1 +plt.xticks(np.arange(0, 1.1, 0.1)) +plt.xlim(0, 1) # 强制显示范围在 0 到 1 之间 + +plt.yscale("log") +plt.xlabel("CDF (Probability)") +plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)") plt.title( - f"Freshness CDF Distribution ({BIN_SECONDS}-Second Sampled, Skip {SKIP_SECONDS}s)" + f"Inverted Freshness CDF ({BIN_SECONDS}-Second Sampled, Skip {SKIP_SECONDS}s)" ) + +plt.grid(True, which="both", ls="-", alpha=0.3) plt.legend() plt.tight_layout() -plt.savefig("freshness_cdf_variable_bin.png") +plt.savefig("freshness_cdf_fixed_ticks.png") plt.close() print("图已生成: freshness_over_time_variable_bin.png, freshness_cdf_variable_bin.png") diff --git a/perf_rate.py b/perf_rate.py index 09a92d7..988662a 100644 --- a/perf_rate.py +++ b/perf_rate.py @@ -7,104 +7,110 @@ # 配置 CSV 文件 和 标签 ########################################## csv_files = { - "150k_2node": "tmp/rate150k_6.csv", + "10k": "resulti7i/10k_rate_2.csv", + "20k": "resulti7i/20k_rate_2.csv", + "30k": "resulti7i/30k_rate_2.csv", + "40k": "resulti7i/40k_rate_2.csv", + "50k": "resulti7i/50k_rate.csv", + "60k": "resulti7i/60k_rate_2.csv", + "80k": "resulti7i/80k_rate_2.csv", + "100k": "resulti7i/100k_rate_2.csv", } -# CSV 文件列名 (无表头) COL_NAMES = ["time", "rows", "txns", "debezium", "serdRows", "serdTxs"] PLOT_COL = "rows" -MAX_SECONDS = 1800 # 截取前多少秒的数据 -SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) -BIN_SECONDS = 60 # 平均窗口(秒) +MAX_SECONDS = 1800 +SKIP_SECONDS = 10 +BIN_SECONDS = 20 ########################################## # 加载并处理数据 ########################################## data = {} for label, path in csv_files.items(): - # 假设文件是逗号分隔 - df = pd.read_csv( - path, - header=None, - names=COL_NAMES, - sep=',' # 假设是逗号分隔,如果不是请修改 - ) - - # --- 时间戳处理 --- - # 1. 解析 HH:MM:SS 时间字符串 - # 2. 由于没有日期,将其与当前日期合并,形成完整的 datetime 对象 (ts) + print(f"正在处理: {label} -> {path}") + + # 1. 加载数据 + df = pd.read_csv(path, header=None, names=COL_NAMES, sep=',') + + # 2. 【核心修改】处理时间戳并跳过格式不对的行 + # errors='coerce' 会将无法解析的格式转为 NaT df["ts"] = pd.to_datetime(df["time"], format="%H:%M:%S", errors='coerce') + + # 剔除无法解析时间的行 (NaT) + initial_count = len(df) + df = df.dropna(subset=["ts"]).copy() + if len(df) < initial_count: + print(f" 注意: 跳过了 {initial_count - len(df)} 行格式不正确的数据") + + # 合并日期 df["ts"] = df["ts"].dt.time.apply(lambda x: datetime.combine(date.today(), x)) - # 相对秒 (使用第一个时间戳作为 t0) + # 3. 计算相对时间 + df = df.sort_values("ts") # 确保时间有序 t0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t0).dt.total_seconds() - # 跳过前 SKIP_SECONDS 秒 - df = df[df["sec"] >= SKIP_SECONDS].copy() # 使用 .copy() 避免 SettingWithCopyWarning + # 4. 过滤时间范围 + df = df[df["sec"] >= SKIP_SECONDS].copy() + if df.empty: + print(f" 警告: {label} 在跳过 {SKIP_SECONDS}s 后没有剩余数据") + continue - # 重新计算时间(所有曲线从 0 秒开始对齐) t_new0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t_new0).dt.total_seconds() - - # 只取前 MAX_SECONDS 秒 df = df[df["sec"] <= MAX_SECONDS] - # 可调平均窗口采样 (将时间序列设为索引进行重采样) - df = df.set_index("ts") - df_bin = df.resample(f"{BIN_SECONDS}s").mean().reset_index() - - # 对齐横轴(时间序列) - df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() + # 5. 【核心修改】重采样聚合 + # 设置索引前,先确保 PLOT_COL 等数值列是 numeric 类型 + # 这样可以防止其他列中混入字符串导致 mean() 失败 + for col in ["rows", "txns", "debezium", "serdRows", "serdTxs"]: + df[col] = pd.to_numeric(df[col], errors='coerce') - data[label] = df_bin + df = df.set_index("ts") + + # 只对数值列进行 mean 运算,忽略字符串列(如 time 列) + df_bin = df.resample(f"{BIN_SECONDS}s").mean(numeric_only=True).reset_index() + # 6. 对齐横轴 + if not df_bin.empty: + df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() + data[label] = df_bin ########################################## -# 图 1:按可调窗口采样的时间序列波动 +# 图 1:时间序列波动 ########################################## plt.figure(figsize=(10, 5)) - for label, df in data.items(): - # 绘制选定的列 plt.plot(df["bin_sec"], df[PLOT_COL], label=label) plt.xlabel("Time (sec)") plt.ylabel(f"{PLOT_COL} ({BIN_SECONDS}s average)") -# plt.yscale("log") # 绘制 "rows" 等计数时,通常不使用对数坐标 -plt.title( - f"{PLOT_COL} Over Time ({BIN_SECONDS}-Second Avg, " - f"Skip {SKIP_SECONDS}s, First {MAX_SECONDS}s)" -) +plt.title(f"{PLOT_COL} Over Time ({BIN_SECONDS}s Avg)") plt.legend() plt.grid(True, linestyle='--', alpha=0.7) plt.tight_layout() -plt.savefig(f"{PLOT_COL}_over_time_variable_bin.png") +plt.savefig(f"rate_{PLOT_COL}_over_time_variable_bin.png") plt.close() - ########################################## -# 图 2:CDF(同样使用平均窗口后的数据) +# 图 2:CDF ########################################## plt.figure(figsize=(10, 5)) - for label, df in data.items(): - # 对选定的列进行 CDF 计算 vals = np.sort(df[PLOT_COL].dropna()) - y = np.linspace(0, 1, len(vals)) - plt.plot(vals, y, label=label) + if len(vals) > 0: + y = np.linspace(0, 1, len(vals)) + plt.plot(vals, y, label=label) -# plt.xscale("log") # 绘制 "rows" 等计数时,通常不使用对数坐标 plt.xlabel(f"{PLOT_COL} ({BIN_SECONDS}s average)") plt.ylabel("CDF") -plt.title( - f"{PLOT_COL} CDF Distribution ({BIN_SECONDS}-Second Sampled, Skip {SKIP_SECONDS}s)" -) +plt.title(f"{PLOT_COL} CDF Distribution") plt.legend() plt.grid(True, linestyle='--', alpha=0.7) plt.tight_layout() -plt.savefig(f"{PLOT_COL}_cdf_variable_bin.png") +plt.savefig(f"rate_{PLOT_COL}_cdf_variable_bin.png") plt.close() -print(f"图已生成: {PLOT_COL}_over_time_variable_bin.png, {PLOT_COL}_cdf_variable_bin.png") \ No newline at end of file +print(f"\n全部完成! 图已生成。") \ No newline at end of file From 7fbf966e97c6676120badb84a8d3abdbdc8bb47b Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 24 Dec 2025 13:24:48 +0000 Subject: [PATCH 51/53] Reformat Code --- conf/jvm.conf | 2 +- conf/pixels-sink.aws.properties | 26 ++-- conf/pixels-sink.pg.properties | 78 ++++++---- perf_freshness.py | 70 ++++++--- perf_rate.py | 18 +-- perf_web_monitor.py | 2 +- .../sink/config/PixelsSinkConstants.java | 2 +- .../pixels/sink/source/SinkSourceFactory.java | 3 + .../{ => engine}/PixelsDebeziumConsumer.java | 2 +- .../source/{ => engine}/SinkEngineSource.java | 3 +- .../source/{ => kafka}/SinkKafkaSource.java | 3 +- .../AbstractMemorySinkStorageSource.java | 147 ++++++++++++++++++ .../AbstractReaderSinkStorageSource.java | 126 +++++++++++++++ .../AbstractSinkStorageSource.java | 127 +++------------ .../FasterSinkStorageSource.java | 5 +- .../LegacySinkStorageSource.java | 14 +- .../writer/retina/RetinaServiceProxy.java | 13 +- .../writer/retina/TableCrossTxWriter.java | 5 +- .../pixels/sink/writer/TestRetinaWriter.java | 4 +- 19 files changed, 449 insertions(+), 201 deletions(-) rename src/main/java/io/pixelsdb/pixels/sink/source/{ => engine}/PixelsDebeziumConsumer.java (99%) rename src/main/java/io/pixelsdb/pixels/sink/source/{ => engine}/SinkEngineSource.java (96%) rename src/main/java/io/pixelsdb/pixels/sink/source/{ => kafka}/SinkKafkaSource.java (96%) create mode 100644 src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java create mode 100644 src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java rename src/main/java/io/pixelsdb/pixels/sink/source/{ => storage}/AbstractSinkStorageSource.java (59%) rename src/main/java/io/pixelsdb/pixels/sink/source/{ => storage}/FasterSinkStorageSource.java (89%) rename src/main/java/io/pixelsdb/pixels/sink/source/{ => storage}/LegacySinkStorageSource.java (95%) diff --git a/conf/jvm.conf b/conf/jvm.conf index 06600bc..28f01e7 100644 --- a/conf/jvm.conf +++ b/conf/jvm.conf @@ -4,7 +4,7 @@ -Duser.timezone=UTC -Xms8g --Xmx45g +-Xmx90g -XX:+UseG1GC -XX:MaxGCPauseMillis=200 diff --git a/conf/pixels-sink.aws.properties b/conf/pixels-sink.aws.properties index e18dbc2..bf29d0a 100644 --- a/conf/pixels-sink.aws.properties +++ b/conf/pixels-sink.aws.properties @@ -1,24 +1,24 @@ # engine | kafka | storage -sink.datasource=storage +sink.datasource=engine # -1 means no limit, Only implement in retina sink mode yet -sink.datasource.rate.limit=100000 +sink.datasource.rate.limit=1000000 # Sink Config: retina | csv | proto | flink | none -sink.mode=retina -sink.retina.client=8 +sink.mode=proto +sink.retina.client=1 sink.retina.log.queue=false ## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature sink.trans.mode=batch sink.monitor.report.enable=true -sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/100k_rate_2.csv -sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/100k_freshness_2.csv +sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i_100/100k_rate.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i_100/100k_fresh.csv # trino for freshness query -trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x +trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf100x # trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x trino.user=pixels trino.password=password -trino.parallel=8 +trino.parallel=4 # row or txn or embed -sink.monitor.freshness.level=embed +sink.monitor.freshness.level=row sink.monitor.freshness.embed.warmup=10 sink.monitor.freshness.embed.static=false sink.monitor.freshness.embed.snapshot=true @@ -57,8 +57,10 @@ sink.commit.batch.size=10 sink.commit.batch.worker=32 sink.commit.batch.delay=3000 ## Proto Config -sink.proto.dir=file:///home/ubuntu/disk1/hybench/ -sink.proto.data=hybench10_10 +sink.proto.dir=file:///home/ubuntu/disk2/hybench/ +sink.proto.data=hybench1000_1 +# sink.proto.data=hybench100_3 +# sink.proto.data=hybench10_10 sink.proto.maxRecords=100000 ## Flink Config sink.flink.server.port=9091 @@ -92,7 +94,7 @@ debezium.database.hostname=realtime-pg-2 debezium.database.port=5432 debezium.database.user=pixels debezium.database.password=pixels_realtime_crud -debezium.database.dbname=pixels_bench_sf10x +debezium.database.dbname=pixels_bench debezium.plugin.name=pgoutput debezium.database.server.id=1 debezium.schema.include.list=public diff --git a/conf/pixels-sink.pg.properties b/conf/pixels-sink.pg.properties index 136ca46..effc171 100644 --- a/conf/pixels-sink.pg.properties +++ b/conf/pixels-sink.pg.properties @@ -1,14 +1,37 @@ # engine | kafka | storage -sink.datasource=storage -# Sink Config: retina | csv | proto | none -sink.mode=none +sink.datasource=engine +# -1 means no limit, Only implement in retina sink mode yet +sink.datasource.rate.limit=100000 +# Sink Config: retina | csv | proto | flink | none +sink.mode=retina +sink.retina.client=8 +sink.retina.log.queue=false +## batch or single or record, batch is recommend. record is faster, but doesn't have ACID feature +sink.trans.mode=batch +sink.monitor.report.enable=true +sink.monitor.report.file=/home/ubuntu/pixels-sink/resulti7i/100k_rate_2.csv +sink.monitor.freshness.file=/home/ubuntu/pixels-sink/resulti7i/100k_freshness_2.csv +# trino for freshness query +trino.url=jdbc:trino://realtime-kafka-2:8080/pixels/pixels_bench_sf10x +# trino.url=jdbc:trino://realtime-pixels-coordinator:8080/pixels/pixels_bench_sf10x +trino.user=pixels +trino.password=password +trino.parallel=8 +# row or txn or embed +sink.monitor.freshness.level=embed +sink.monitor.freshness.embed.warmup=10 +sink.monitor.freshness.embed.static=false +sink.monitor.freshness.embed.snapshot=true +sink.monitor.freshness.embed.tablelist=loanapps,loantrans +sink.monitor.freshness.verbose=true +sink.monitor.freshness.timestamp=true sink.storage.loop=true # Kafka Config -bootstrap.servers=localhost:29092 +bootstrap.servers=realtime-kafka-2:29092 group.id=3078 auto.offset.reset=earliest key.deserializer=org.apache.kafka.common.serialization.StringDeserializer -#value.deserializer=io.pixelsdb.pixels.sink.deserializer.RowChangeEventAvroDeserializer +#value.deserializer=io.pixelsdb.pixels.writer.deserializer.RowChangeEventAvroDeserializer value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventJsonDeserializer # Topic & Database Config topic.prefix=postgresql.oltp_server @@ -20,39 +43,43 @@ sink.csv.enable_header=false sink.retina.embedded=false # stub or stream sink.retina.mode=stream -#sink.retina.mode=stub +#writer.retina.mode=stub sink.remote.host=localhost sink.remote.port=29422 sink.timeout.ms=5000 -sink.flush.interval.ms=100 -sink.flush.batch.size=100 +sink.flush.interval.ms=50 +sink.flush.batch.size=10 sink.max.retries=3 -## sink commit -sink.commit.batch.size=500 +## writer commit +# sync or async +sink.commit.method=sync +sink.commit.batch.size=10 sink.commit.batch.worker=32 -sink.commit.batch.delay=200 +sink.commit.batch.delay=3000 ## Proto Config -sink.proto.dir=file:///home/antio2/projects/pixels-sink/tmp -sink.proto.data=data -sink.proto.maxRecords=1000000 +sink.proto.dir=file:///home/ubuntu/disk1/hybench/ +sink.proto.data=hybench10_10 +sink.proto.maxRecords=100000 +## Flink Config +sink.flink.server.port=9091 ## Schema Registry sink.registry.url=http://localhost:8080/apis/registry/v2 # Transaction Config transaction.topic.suffix=transaction -#transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.deserializer.TransactionAvroMessageDeserializer +#transaction.topic.value.deserializer=io.pixelsdb.pixels.writer.deserializer.TransactionAvroMessageDeserializer transaction.topic.value.deserializer=io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer -sink.trans.batch.size=200 -## Batch or trans or record -sink.trans.mode=batch +sink.trans.batch.size=100 + # Sink Metrics sink.monitor.enable=true sink.monitor.port=9464 -sink.monitor.report.interval=5000 -sink.monitor.report.file=/home/antio2/projects/pixels-sink/tmp/proto.csv +sink.monitor.report.interval=10000 +sink.monitor.freshness.interval=1000 + # Interact with other rpc sink.rpc.enable=true sink.rpc.mock.delay=20 -# debezium +# debezium engine config debezium.name=testEngine debezium.connector.class=io.debezium.connector.postgresql.PostgresConnector debezium.provide.transaction.metadata=true @@ -61,11 +88,11 @@ debezium.offset.storage.file.filename=/tmp/offsets.dat debezium.offset.flush.interval.ms=60000 debezium.schema.history.internal=io.debezium.storage.file.history.FileSchemaHistory debezium.schema.history.internal.file.filename=/tmp/schemahistory.dat -debezium.database.hostname=localhost +debezium.database.hostname=realtime-pg-2 debezium.database.port=5432 debezium.database.user=pixels -debezium.database.password=password -debezium.database.dbname=pixels_bench_sf1x +debezium.database.password=pixels_realtime_crud +debezium.database.dbname=pixels_bench_sf10x debezium.plugin.name=pgoutput debezium.database.server.id=1 debezium.schema.include.list=public @@ -76,5 +103,4 @@ debezium.topic.prefix=postgresql.oltp_server debezium.transforms=topicRouting debezium.transforms.topicRouting.type=org.apache.kafka.connect.transforms.RegexRouter debezium.transforms.topicRouting.regex=postgresql\\.oltp_server\\.public\\.(.*) -debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf1x.$1 - +debezium.transforms.topicRouting.replacement=postgresql.oltp_server.pixels_bench_sf10x.$1 diff --git a/perf_freshness.py b/perf_freshness.py index b8bcbdd..f32f572 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -1,21 +1,27 @@ import pandas as pd import matplotlib.pyplot as plt import numpy as np +import seaborn as sns ########################################## # 配置 CSV 文件 和 标签 ########################################## csv_files = { # "10k_2": "resulti7i/10k_freshness.csv", - "10k": "resulti7i/10k_freshness_2.csv", + # "10k": "resulti7i/10k_freshness_2.csv", # "20k": "resulti7i/20k_freshness.csv", - "20k": "resulti7i/20k_freshness_2.csv", - "30k": "resulti7i/30k_freshness_2.csv", - "40k": "resulti7i/40k_freshness_2.csv", - "50k": "resulti7i/50k_freshness.csv", - "60k": "resulti7i/60k_freshness_2.csv", - "80k": "resulti7i/80k_freshness_2.csv", - "100k": "resulti7i/100k_freshness_2.csv", + # "20k": "resulti7i/20k_freshness_2.csv", + # "30k": "resulti7i/30k_freshness_2.csv", + # "40k": "resulti7i/40k_freshness_2.csv", + # "50k": "resulti7i/50k_freshness.csv", + # "60k": "resulti7i/60k_freshness_2.csv", + # "80k": "resulti7i/80k_freshness_2.csv", + # "10k": "resulti7i_100/10k_fresh.csv", + # "20k": "resulti7i_100/20k_fresh.csv", + # # "30k": "resulti7i_100/30k_fresh.csv", + # "40k": "resulti7i_100/40k_fresh.csv", + # "60k": "resulti7i_100/60k_fresh.csv", + "100k": "resulti7i_100/100k_fresh.csv", } # csv_files = { # "Query Transaction": "tmp/i7i_2k_dec_freshness.csv", @@ -25,7 +31,7 @@ # } MAX_SECONDS = 1800 # 截取前多少秒的数据 SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) -BIN_SECONDS = 10 # 平均窗口(秒) +BIN_SECONDS = 180 # 平均窗口(秒) MAX_FRESHNESS = 500000 # 过滤初始warmup时的无用数据 ########################################## # 加载并处理数据 @@ -63,23 +69,47 @@ ########################################## -# 图 1:按可调窗口采样的时间序列波动 +# 图 1:平滑美化后的时间序列波动 ########################################## -plt.figure(figsize=(10, 5)) +# 设置整体风格,whitegrid 看起来非常干净专业 +sns.set_theme(style="whitegrid") + +plt.figure(figsize=(12, 6)) # 稍微拉宽,适合看时间趋势 for label, df in data.items(): - plt.plot(df["bin_sec"], df["freshness"], label=label) + # 确保数据已排序 + df_plot = df.sort_values("bin_sec") + + # 方案 A: 增加线宽和抗锯齿,使用半透明填充让多条线重叠时更容易分辨 + line, = plt.plot( + df_plot["bin_sec"], + df_plot["freshness"], + label=label, + linewidth=1.8, + alpha=0.9, + antialiased=True + ) + + +# 坐标轴美化 +plt.xlabel("Time (sec)", fontsize=11, fontweight='bold') +plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)", fontsize=11, fontweight='bold') + +# 去掉上方和右侧的边框,让图表更清爽 +sns.despine() -plt.xlabel("Time (sec)") -plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)") -# plt.yscale("log") plt.title( - f"Freshness Over Time ({BIN_SECONDS}-Second Avg, " - f"Skip {SKIP_SECONDS}s, First {MAX_SECONDS}s)" + f"Freshness Oscillations\n({BIN_SECONDS}s Binning, Skip {SKIP_SECONDS}s)", + fontsize=13, + pad=15 ) -plt.legend() + +# 调整图例到外面或右上角,防止遮挡曲线 +plt.legend(bbox_to_anchor=(1.05, 1), loc='upper left', borderaxespad=0.) + +plt.grid(True, which="major", ls="-", alpha=0.4) plt.tight_layout() -plt.savefig("freshness_over_time_variable_bin.png") +plt.savefig("freshness_over_time_smooth.png", dpi=300) # 提高分辨率 plt.close() @@ -99,7 +129,7 @@ plt.xticks(np.arange(0, 1.1, 0.1)) plt.xlim(0, 1) # 强制显示范围在 0 到 1 之间 -plt.yscale("log") +# plt.yscale("log") plt.xlabel("CDF (Probability)") plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)") plt.title( diff --git a/perf_rate.py b/perf_rate.py index 988662a..55f49fc 100644 --- a/perf_rate.py +++ b/perf_rate.py @@ -7,14 +7,14 @@ # 配置 CSV 文件 和 标签 ########################################## csv_files = { - "10k": "resulti7i/10k_rate_2.csv", - "20k": "resulti7i/20k_rate_2.csv", - "30k": "resulti7i/30k_rate_2.csv", - "40k": "resulti7i/40k_rate_2.csv", - "50k": "resulti7i/50k_rate.csv", - "60k": "resulti7i/60k_rate_2.csv", - "80k": "resulti7i/80k_rate_2.csv", - "100k": "resulti7i/100k_rate_2.csv", + # "10k": "resulti7i/10k_rate_2.csv", + # "20k": "resulti7i/20k_rate_2.csv", + # "30k": "resulti7i/30k_rate_2.csv", + # "40k": "resulti7i/40k_rate_2.csv", + # "50k": "resulti7i/50k_rate.csv", + # "60k": "resulti7i/60k_rate_2.csv", + # "80k": "resulti7i/80k_rate_2.csv", + "100k": "resulti7i_100/100k_rate.csv", } COL_NAMES = ["time", "rows", "txns", "debezium", "serdRows", "serdTxs"] @@ -22,7 +22,7 @@ MAX_SECONDS = 1800 SKIP_SECONDS = 10 -BIN_SECONDS = 20 +BIN_SECONDS = 60 ########################################## # 加载并处理数据 diff --git a/perf_web_monitor.py b/perf_web_monitor.py index b7cbc29..2ae816f 100644 --- a/perf_web_monitor.py +++ b/perf_web_monitor.py @@ -5,7 +5,7 @@ from time import time # Configuration -DATA_DIR = "/home/ubuntu/pixels-sink/tmp" +DATA_DIR = "/home/ubuntu/pixels-sink/resulti7i_100" # DATA_DIR = "/home/antio2/projects/pixels-sink/tmp" PORT = 8083 CACHE_TTL = 5 # seconds diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java index adeb4eb..5b6e1f6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java @@ -25,7 +25,7 @@ public final class PixelsSinkConstants public static final String ROW_RECORD_KAFKA_PROP_FACTORY = "row-record"; public static final String TRANSACTION_KAFKA_PROP_FACTORY = "transaction"; public static final int MONITOR_NUM = 2; - public static final int MAX_QUEUE_SIZE = 10_000; + public static final int MAX_QUEUE_SIZE = 1_000; public static final String SNAPSHOT_TX_PREFIX = "SNAPSHOT-"; private PixelsSinkConstants() diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java index dece1f8..207edb8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java @@ -23,6 +23,9 @@ import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.source.engine.SinkEngineSource; +import io.pixelsdb.pixels.sink.source.kafka.SinkKafkaSource; +import io.pixelsdb.pixels.sink.source.storage.FasterSinkStorageSource; public class SinkSourceFactory { diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java similarity index 99% rename from src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java rename to src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java index f936f7f..933473a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.source; +package io.pixelsdb.pixels.sink.source.engine; import io.debezium.engine.DebeziumEngine; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java similarity index 96% rename from src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java rename to src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java index 8c9d548..a692ea8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkEngineSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java @@ -18,13 +18,14 @@ * . */ -package io.pixelsdb.pixels.sink.source; +package io.pixelsdb.pixels.sink.source.engine; import io.debezium.embedded.Connect; import io.debezium.engine.DebeziumEngine; import io.debezium.engine.RecordChangeEvent; import io.debezium.engine.format.ChangeEventFormat; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; +import io.pixelsdb.pixels.sink.source.SinkSource; import org.apache.kafka.connect.source.SourceRecord; import java.util.Properties; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java similarity index 96% rename from src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java rename to src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java index c2e696b..c82fe1e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkKafkaSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.source; +package io.pixelsdb.pixels.sink.source.kafka; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; @@ -27,6 +27,7 @@ import io.pixelsdb.pixels.sink.processor.MonitorThreadManager; import io.pixelsdb.pixels.sink.processor.TopicProcessor; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; +import io.pixelsdb.pixels.sink.source.SinkSource; import java.util.Properties; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java new file mode 100644 index 0000000..fc8b857 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java @@ -0,0 +1,147 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +package io.pixelsdb.pixels.sink.source.storage; + +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; +import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.core.utils.Pair; +import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; +import io.pixelsdb.pixels.sink.provider.ProtoType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; + +public abstract class AbstractMemorySinkStorageSource extends AbstractSinkStorageSource +{ + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractMemorySinkStorageSource.class); + + // All preloaded records, order preserved + // key + value buffer + private final List> preloadedRecords = new ArrayList<>(); + + @Override + public void start() + { + this.running.set(true); + this.transactionProcessorThread.start(); + this.transactionProviderThread.start(); + try + { + /* ===================================================== + * 1. Initialization phase: preload all ByteBuffers + * ===================================================== */ + for (String file : files) + { + Storage.Scheme scheme = Storage.Scheme.fromPath(file); + LOGGER.info("Preloading file {}", file); + + PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); + readers.add(reader); + + while (true) + { + int key; + int valueLen; + + try + { + key = reader.readInt(ByteOrder.BIG_ENDIAN); + valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + } catch (IOException eof) + { + // Reached end of file + break; + } + // Synchronous read and copy to heap buffer + ByteBuffer valueBuffer = reader.readFully(valueLen); + // Store into a single global array + preloadedRecords.add(new Pair<>(key, valueBuffer)); + } + } + + LOGGER.info("Preload finished, total records = {}", preloadedRecords.size()); + + /* ===================================================== + * Phase 2: Runtime loop + * Queue initialization, consumer startup, and feeding + * are done together in this phase + * ===================================================== */ + do + { + for (Pair record : preloadedRecords) + { + int key = record.getLeft(); + ByteBuffer buffer = record.getRight(); + + // Lazily create queue + BlockingQueue, Integer>> queue = + queueMap.computeIfAbsent( + key, + k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE) + ); + + // Lazily start consumer thread + consumerThreads.computeIfAbsent(key, k -> + { + ProtoType protoType = getProtoType(k); + Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); + t.setName("consumer-" + k); + t.start(); + return t; + }); + + ProtoType protoType = getProtoType(key); + if (protoType == ProtoType.ROW) + { + sourceRateLimiter.acquire(1); + } + + // Use completed future to keep consumer logic unchanged + CompletableFuture future = + CompletableFuture.completedFuture(buffer); + + queue.put(new Pair<>(future, loopId)); + } + ++loopId; + } while (storageLoopEnabled && isRunning()); + } catch (IOException | IndexOutOfBoundsException e) + { + throw new RuntimeException(e); + } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + } + finally + { + clean(); + } + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java new file mode 100644 index 0000000..2055cc3 --- /dev/null +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java @@ -0,0 +1,126 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +package io.pixelsdb.pixels.sink.source.storage; + +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; +import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.core.utils.Pair; +import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; +import io.pixelsdb.pixels.sink.provider.ProtoType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; + +public abstract class AbstractReaderSinkStorageSource extends AbstractSinkStorageSource +{ + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractReaderSinkStorageSource.class); + + @Override + public void start() + { + this.running.set(true); + this.transactionProcessorThread.start(); + this.transactionProviderThread.start(); + for (String file : files) + { + Storage.Scheme scheme = Storage.Scheme.fromPath(file); + LOGGER.info("Start read from file {}", file); + PhysicalReader reader; + try + { + reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); + } catch (IOException e) + { + throw new RuntimeException(e); + } + readers.add(reader); + } + do + { + for (PhysicalReader reader : readers) + { + LOGGER.info("Start Read {}", reader.getPath()); + long offset = 0; + while (true) + { + try + { + int key, valueLen; + reader.seek(offset); + try + { + key = reader.readInt(ByteOrder.BIG_ENDIAN); + valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); + } catch (IOException e) + { + // EOF + break; + } + + ProtoType protoType = getProtoType(key); + offset += Integer.BYTES * 2; + CompletableFuture valueFuture = reader.readAsync(offset, valueLen) + .thenApply(this::copyToHeap) + .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); + // move offset for next record + offset += valueLen; + + + // Get or create queue + BlockingQueue, Integer>> queue = + queueMap.computeIfAbsent(key, + k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE)); + + // Put future in queue + if(protoType.equals(ProtoType.ROW)) + { + sourceRateLimiter.acquire(1); + } + queue.put(new Pair<>(valueFuture, loopId)); + // Start consumer thread if not exists + consumerThreads.computeIfAbsent(key, k -> + { + Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); + t.setName("consumer-" + key); + t.start(); + return t; + }); + } catch (IOException | InterruptedException e) + { + break; + } + } + } + ++loopId; + } while (storageLoopEnabled && isRunning()); + + clean(); + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java similarity index 59% rename from src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java rename to src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java index f0cd58f..66702df 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java @@ -18,21 +18,18 @@ * . */ -package io.pixelsdb.pixels.sink.source; +package io.pixelsdb.pixels.sink.source.storage; import io.pixelsdb.pixels.common.physical.PhysicalReader; -import io.pixelsdb.pixels.common.physical.PhysicalReaderUtil; -import io.pixelsdb.pixels.common.physical.Storage; import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TransactionProcessor; import io.pixelsdb.pixels.sink.provider.ProtoType; import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; import io.pixelsdb.pixels.sink.provider.TransactionEventStorageLoopProvider; -import io.pixelsdb.pixels.sink.provider.TransactionEventStorageProvider; +import io.pixelsdb.pixels.sink.source.SinkSource; import io.pixelsdb.pixels.sink.util.EtcdFileRegistry; import io.pixelsdb.pixels.sink.util.FlushRateLimiter; import io.pixelsdb.pixels.sink.util.MetricsFacade; @@ -41,7 +38,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.ByteOrder; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -58,19 +54,19 @@ public abstract class AbstractSinkStorageSource implements SinkSource protected final EtcdFileRegistry etcdFileRegistry; protected final List files; protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); - private final Map consumerThreads = new ConcurrentHashMap<>(); + protected final Map consumerThreads = new ConcurrentHashMap<>(); private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - private final Map, Integer>>> queueMap = new ConcurrentHashMap<>(); + protected final Map, Integer>>> queueMap = new ConcurrentHashMap<>(); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final TableProviderAndProcessorPipelineManager> tablePipelineManager = new TableProviderAndProcessorPipelineManager<>(); - private final boolean storageLoopEnabled; - private final FlushRateLimiter sourceRateLimiter; + protected final boolean storageLoopEnabled; + protected final FlushRateLimiter sourceRateLimiter; protected TransactionEventStorageLoopProvider> transactionEventProvider; protected TransactionProcessor transactionProcessor; protected Thread transactionProviderThread; protected Thread transactionProcessorThread; - private int loopId = 0; - + protected int loopId = 0; + protected List readers = new ArrayList<>(); protected AbstractSinkStorageSource() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); @@ -90,130 +86,51 @@ protected AbstractSinkStorageSource() abstract ProtoType getProtoType(int i); - protected void handleTransactionSourceRecord(ByteBuffer record, Integer loopId) - { - transactionEventProvider.putTransRawEvent(new Pair<>(record, loopId)); - } - - @Override - public void start() + protected void clean() { - this.running.set(true); - this.transactionProcessorThread.start(); - this.transactionProviderThread.start(); - List readers = new ArrayList<>(); - for (String file : files) - { - Storage.Scheme scheme = Storage.Scheme.fromPath(file); - LOGGER.info("Start read from file {}", file); - PhysicalReader reader = null; - try - { - reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); - } catch (IOException e) - { - throw new RuntimeException(e); - } - readers.add(reader); - } - do - { - for (PhysicalReader reader : readers) - { - LOGGER.info("Start Read {}", reader.getPath()); - long offset = 0; - while (true) - { - try - { - int key, valueLen; - reader.seek(offset); - try - { - key = reader.readInt(ByteOrder.BIG_ENDIAN); - valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) - { - // EOF - break; - } - - ProtoType protoType = getProtoType(key); - offset += Integer.BYTES * 2; - CompletableFuture valueFuture = reader.readAsync(offset, valueLen) - .thenApply(this::copyToHeap) - .thenApply(buf -> buf.order(ByteOrder.BIG_ENDIAN)); - // move offset for next record - offset += valueLen; - - - // Get or create queue - BlockingQueue, Integer>> queue = - queueMap.computeIfAbsent(key, - k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE)); - - // Put future in queue - if(protoType.equals(ProtoType.ROW)) - { - sourceRateLimiter.acquire(1); - } - queue.put(new Pair<>(valueFuture, loopId)); - // Start consumer thread if not exists - consumerThreads.computeIfAbsent(key, k -> - { - Thread t = new Thread(() -> consumeQueue(k, queue, protoType)); - t.setName("consumer-" + key); - t.start(); - return t; - }); - } catch (IOException | InterruptedException e) - { - break; - } - } - } - ++loopId; - } while (storageLoopEnabled && isRunning()); - - // signal all queues to stop queueMap.values().forEach(q -> { try { q.put(new Pair<>(POISON_PILL, loopId)); - } catch (InterruptedException e) + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); - // wait all consumers to finish consumerThreads.values().forEach(t -> { try { t.join(); - } catch (InterruptedException e) + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); - // close all readers for (PhysicalReader reader : readers) { try { reader.close(); - } catch (IOException e) + } + catch (IOException e) { - throw new RuntimeException(e); + LOGGER.warn("Failed to close reader", e); } } + } + protected void handleTransactionSourceRecord(ByteBuffer record, Integer loopId) + { + transactionEventProvider.putTransRawEvent(new Pair<>(record, loopId)); } - private void consumeQueue(int key, BlockingQueue, Integer>> queue, ProtoType protoType) + protected void consumeQueue(int key, BlockingQueue, Integer>> queue, ProtoType protoType) { try { @@ -243,7 +160,7 @@ private void consumeQueue(int key, BlockingQueue. */ -package io.pixelsdb.pixels.sink.source; +package io.pixelsdb.pixels.sink.source.storage; import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.sink.provider.ProtoType; +import io.pixelsdb.pixels.sink.source.SinkSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +35,7 @@ * @author: AntiO2 * @date: 2025/10/5 11:43 */ -public class FasterSinkStorageSource extends AbstractSinkStorageSource implements SinkSource +public class FasterSinkStorageSource extends AbstractMemorySinkStorageSource implements SinkSource { private static final Logger LOGGER = LoggerFactory.getLogger(FasterSinkStorageSource.class); static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java similarity index 95% rename from src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java rename to src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java index baa2153..e8fd71e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/LegacySinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java @@ -18,7 +18,7 @@ * . */ -package io.pixelsdb.pixels.sink.source; +package io.pixelsdb.pixels.sink.source.storage; import io.pixelsdb.pixels.common.metadata.SchemaTableName; @@ -31,6 +31,7 @@ import io.pixelsdb.pixels.sink.provider.ProtoType; import io.pixelsdb.pixels.sink.provider.TableProviderAndProcessorPipelineManager; import io.pixelsdb.pixels.sink.provider.TransactionEventEngineProvider; +import io.pixelsdb.pixels.sink.source.SinkSource; import io.pixelsdb.pixels.sink.util.MetricsFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +49,7 @@ * @date: 2025/10/5 11:43 */ @Deprecated -public class LegacySinkStorageSource extends AbstractSinkStorageSource implements SinkSource +public class LegacySinkStorageSource extends AbstractReaderSinkStorageSource implements SinkSource { private static final Logger LOGGER = LoggerFactory.getLogger(LegacySinkStorageSource.class); static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); @@ -171,15 +172,6 @@ public void start() }); } - private ByteBuffer copyToHeap(ByteBuffer directBuffer) - { - ByteBuffer duplicate = directBuffer.duplicate(); - ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); - heapBuffer.put(duplicate); - heapBuffer.flip(); - return heapBuffer; - } - private void consumeQueue(SchemaTableName key, BlockingQueue> queue, ProtoType protoType) { try diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java index ef00e9c..eeb694a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java @@ -21,6 +21,7 @@ package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.node.BucketCache; import io.pixelsdb.pixels.common.retina.RetinaService; import io.pixelsdb.pixels.common.utils.RetinaUtils; import io.pixelsdb.pixels.retina.RetinaProto; @@ -48,7 +49,7 @@ public class RetinaServiceProxy private final RetinaService retinaService; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private RetinaService.StreamHandler retinaStream = null; - + private final int vNodeId; public RetinaServiceProxy(int bucketId) { if (bucketId == -1) @@ -67,6 +68,8 @@ public RetinaServiceProxy(int bucketId) { retinaStream = null; } + + this.vNodeId = BucketCache.getInstance().getRetinaNodeInfoByBucketId(bucketId).getVirtualNodeId(); } public boolean writeTrans(String schemaName, List tableUpdateData) @@ -75,7 +78,7 @@ public boolean writeTrans(String schemaName, List t { try { - retinaService.updateRecord(schemaName, tableUpdateData); + retinaService.updateRecord(schemaName, vNodeId, tableUpdateData); } catch (RetinaException e) { e.printStackTrace(); @@ -85,7 +88,7 @@ public boolean writeTrans(String schemaName, List t { try { - retinaStream.updateRecord(schemaName, tableUpdateData); + retinaStream.updateRecord(schemaName, vNodeId, tableUpdateData); } catch (RetinaException e) { e.printStackTrace(); @@ -102,7 +105,7 @@ public boolean writeTrans(String schemaName, List t { try { - retinaService.updateRecord(schemaName, tableUpdateData); + retinaService.updateRecord(schemaName, vNodeId, tableUpdateData); } catch (RetinaException e) { e.printStackTrace(); @@ -112,7 +115,7 @@ public boolean writeTrans(String schemaName, List t { try { - return retinaStream.updateRecord(schemaName, tableUpdateData); + return retinaStream.updateRecord(schemaName, vNodeId, tableUpdateData); } catch (RetinaException e) { e.printStackTrace(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index a1da79a..8d35cd7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -47,13 +47,11 @@ public class TableCrossTxWriter extends TableWriter private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); private final int flushBatchSize; protected final ReentrantLock writeLock = new ReentrantLock(); - private final int bucketId; public TableCrossTxWriter(String t, int bucketId) { super(t, bucketId); flushBatchSize = config.getFlushBatchSize(); - this.bucketId = bucketId; } /** @@ -118,7 +116,8 @@ public void flush(List batch) { tableUpdateData.add(tableUpdateDataItem.build()); } - CompletableFuture updateRecordResponseCompletableFuture = delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); + CompletableFuture updateRecordResponseCompletableFuture = + delegate.writeBatchAsync(batch.get(0).getSchemaName(), tableUpdateData); updateRecordResponseCompletableFuture.thenAccept( resp -> diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java index d7f848c..db64702 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java @@ -106,7 +106,7 @@ public void insertSingleRecord() throws RetinaException, SinkException, TransExc tableUpdateDataBuilder.addInsertData(insertDataBuilder.build()); } tableUpdateData.add(tableUpdateDataBuilder.build()); - retinaService.updateRecord(schemaName, tableUpdateData); + retinaService.updateRecord(schemaName, 0, tableUpdateData); tableUpdateDataBuilder.setTimestamp(timeStamp); transService.commitTrans(ctx.getTransId(), false); } @@ -177,7 +177,7 @@ public void updateSingleRecord() throws RetinaException, SinkException, TransExc } tableUpdateDataBuilder.setTimestamp(timeStamp); tableUpdateData.add(tableUpdateDataBuilder.build()); - retinaService.updateRecord(schemaName, tableUpdateData); + retinaService.updateRecord(schemaName, 0, tableUpdateData); transService.commitTrans(ctx.getTransId(), false); } From a14e12fb3f1f5ea9ed3aeb85d4aa493e682d244c Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 24 Dec 2025 13:36:57 +0000 Subject: [PATCH 52/53] Reformat Code --- .idea/codeStyles/codeStyleConfig.xml | 5 + .idea/icon.svg | 1 - perf_freshness.py | 78 ++-- perf_rate.py | 54 +-- perf_web_monitor.py | 32 +- .../pixelsdb/pixels/sink/PixelsSinkApp.java | 38 +- .../pixels/sink/PixelsSinkProvider.java | 14 +- .../pixels/sink/config/CommandLineConfig.java | 17 +- .../pixels/sink/config/ConfigKey.java | 5 +- .../pixels/sink/config/ConfigLoader.java | 62 +-- .../pixels/sink/config/PixelsSinkConfig.java | 17 +- .../sink/config/PixelsSinkConstants.java | 8 +- .../sink/config/PixelsSinkDefaultConfig.java | 5 +- .../pixels/sink/config/TransactionConfig.java | 5 +- .../sink/config/factory/KafkaPropFactory.java | 5 +- .../factory/KafkaPropFactorySelector.java | 14 +- .../factory/PixelsSinkConfigFactory.java | 26 +- .../factory/RowRecordKafkaPropFactory.java | 11 +- .../factory/TransactionKafkaPropFactory.java | 8 +- .../pixels/sink/event/RowChangeEvent.java | 173 +++---- .../event/deserializer/DeserializerUtil.java | 56 +-- .../RowChangeEventAvroDeserializer.java | 56 +-- .../RowChangeEventJsonDeserializer.java | 56 +-- .../RowChangeEventStructDeserializer.java | 38 +- .../event/deserializer/RowDataParser.java | 143 ++---- .../deserializer/SchemaDeserializer.java | 89 ++-- .../TransactionAvroMessageDeserializer.java | 23 +- .../TransactionJsonMessageDeserializer.java | 20 +- .../TransactionStructMessageDeserializer.java | 20 +- .../pixels/sink/exception/SinkException.java | 14 +- .../sink/freshness/FreshnessClient.java | 160 +++---- .../sink/freshness/FreshnessHistory.java | 21 +- .../sink/freshness/OneSecondAverage.java | 29 +- .../pixels/sink/metadata/TableMetadata.java | 35 +- .../sink/metadata/TableMetadataRegistry.java | 74 +-- .../sink/processor/MonitorThreadManager.java | 32 +- .../sink/processor/StoppableProcessor.java | 5 +- .../pixels/sink/processor/TableProcessor.java | 28 +- .../pixels/sink/processor/TopicProcessor.java | 140 ++---- .../sink/processor/TransactionProcessor.java | 20 +- .../pixels/sink/provider/EventProvider.java | 92 ++-- .../pixels/sink/provider/ProtoType.java | 20 +- .../provider/TableEventEngineProvider.java | 14 +- .../provider/TableEventKafkaProvider.java | 44 +- .../sink/provider/TableEventProvider.java | 17 +- .../TableEventStorageLoopProvider.java | 33 +- .../provider/TableEventStorageProvider.java | 17 +- ...leProviderAndProcessorPipelineManager.java | 38 +- .../TransactionEventEngineProvider.java | 11 +- .../TransactionEventKafkaProvider.java | 35 +- .../provider/TransactionEventProvider.java | 14 +- .../TransactionEventStorageLoopProvider.java | 14 +- .../TransactionEventStorageProvider.java | 14 +- .../pixels/sink/source/SinkSource.java | 5 +- .../pixels/sink/source/SinkSourceFactory.java | 11 +- .../source/engine/PixelsDebeziumConsumer.java | 41 +- .../sink/source/engine/SinkEngineSource.java | 32 +- .../sink/source/kafka/SinkKafkaSource.java | 14 +- .../AbstractMemorySinkStorageSource.java | 43 +- .../AbstractReaderSinkStorageSource.java | 43 +- .../storage/AbstractSinkStorageSource.java | 85 ++-- .../storage/FasterSinkStorageSource.java | 17 +- .../storage/LegacySinkStorageSource.java | 92 ++-- .../pixels/sink/util/BlockingBoundedMap.java | 56 +-- .../pixels/sink/util/DataTransform.java | 230 +++++---- .../pixelsdb/pixels/sink/util/DateUtil.java | 26 +- .../pixels/sink/util/EtcdFileRegistry.java | 68 +-- .../pixels/sink/util/FlushRateLimiter.java | 48 +- .../pixels/sink/util/LatencySimulator.java | 20 +- .../pixels/sink/util/MetricsFacade.java | 439 ++++++++---------- .../pixels/sink/util/TableCounters.java | 22 +- .../sink/writer/AbstractBucketedWriter.java | 40 +- .../pixels/sink/writer/NoneWriter.java | 129 +++-- .../pixels/sink/writer/PixelsSinkMode.java | 14 +- .../pixels/sink/writer/PixelsSinkWriter.java | 5 +- .../sink/writer/PixelsSinkWriterFactory.java | 28 +- .../pixels/sink/writer/csv/CsvWriter.java | 92 ++-- .../sink/writer/flink/FlinkPollingWriter.java | 47 +- .../flink/PixelsPollingServiceImpl.java | 31 +- .../sink/writer/flink/PollingRpcServer.java | 22 +- .../pixels/sink/writer/proto/ProtoWriter.java | 128 ++--- .../writer/proto/RotatingWriterManager.java | 26 +- .../writer/retina/RetinaBucketDispatcher.java | 9 +- .../writer/retina/RetinaServiceProxy.java | 83 ++-- .../sink/writer/retina/RetinaWriter.java | 83 ++-- .../sink/writer/retina/SinkContext.java | 90 ++-- .../writer/retina/SinkContextManager.java | 146 ++---- .../writer/retina/TableCrossTxWriter.java | 103 ++-- .../retina/TableSingleRecordWriter.java | 56 +-- .../writer/retina/TableSingleTxWriter.java | 47 +- .../sink/writer/retina/TableWriter.java | 212 ++++----- .../sink/writer/retina/TableWriterProxy.java | 32 +- .../sink/writer/retina/TransactionMode.java | 14 +- .../sink/writer/retina/TransactionProxy.java | 143 ++---- 94 files changed, 1788 insertions(+), 3084 deletions(-) create mode 100644 .idea/codeStyles/codeStyleConfig.xml delete mode 100644 .idea/icon.svg diff --git a/.idea/codeStyles/codeStyleConfig.xml b/.idea/codeStyles/codeStyleConfig.xml new file mode 100644 index 0000000..79ee123 --- /dev/null +++ b/.idea/codeStyles/codeStyleConfig.xml @@ -0,0 +1,5 @@ + + + + \ No newline at end of file diff --git a/.idea/icon.svg b/.idea/icon.svg deleted file mode 100644 index ad8442b..0000000 --- a/.idea/icon.svg +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/perf_freshness.py b/perf_freshness.py index f32f572..e5c35ce 100644 --- a/perf_freshness.py +++ b/perf_freshness.py @@ -4,7 +4,7 @@ import seaborn as sns ########################################## -# 配置 CSV 文件 和 标签 +# Configuration: CSV Files and Labels ########################################## csv_files = { # "10k_2": "resulti7i/10k_freshness.csv", @@ -29,105 +29,107 @@ # "Internal Transaction Context": "tmp/i7i_2k_txn_dec_freshness.csv", # "Query Selected Table, Trans Mode": "tmp/i7i_2k_batchtest_dec_freshness_2.csv" # } -MAX_SECONDS = 1800 # 截取前多少秒的数据 -SKIP_SECONDS = 10 # 跳过前多少秒的数据(可调) -BIN_SECONDS = 180 # 平均窗口(秒) -MAX_FRESHNESS = 500000 # 过滤初始warmup时的无用数据 + +MAX_SECONDS = 1800 # Capture data for the first N seconds +SKIP_SECONDS = 10 # Skip the first N seconds (adjustable) +BIN_SECONDS = 180 # Average window (seconds) +MAX_FRESHNESS = 500000 # Filter out useless data during initial warmup ########################################## -# 加载并处理数据 +# Data Loading and Processing ########################################## data = {} for label, path in csv_files.items(): df = pd.read_csv(path, header=None, names=["ts", "freshness"]) - # 转为 datetime + # Convert to datetime df["ts"] = pd.to_datetime(df["ts"], unit="ms") - # 相对秒 + # Relative seconds t0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t0).dt.total_seconds() - # 跳过前 SKIP_SECONDS 秒 + # Skip initial SKIP_SECONDS df = df[df["sec"] >= SKIP_SECONDS] + # Filter by max freshness threshold df = df[df["freshness"] <= MAX_FRESHNESS] - # 重新计算时间(所有曲线从 0 秒开始对齐) + # Recalculate time (align all curves to start at 0 seconds) t_new0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t_new0).dt.total_seconds() - # 只取前 MAX_SECONDS 秒 + # Limit to MAX_SECONDS df = df[df["sec"] <= MAX_SECONDS] - # 可调平均窗口采样 + # Sample using an adjustable averaging window df_bin = df.resample(f"{BIN_SECONDS}s", on="ts").mean().reset_index() - # 对齐横轴(时间序列) + # Align horizontal axis (time series) df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() data[label] = df_bin ########################################## -# 图 1:平滑美化后的时间序列波动 +# Plot 1: Smoothed/Beautified Time Series Oscillations ########################################## -# 设置整体风格,whitegrid 看起来非常干净专业 -sns.set_theme(style="whitegrid") +# Set overall style; whitegrid looks clean and professional +sns.set_theme(style="whitegrid") -plt.figure(figsize=(12, 6)) # 稍微拉宽,适合看时间趋势 +plt.figure(figsize=(12, 6)) # Slightly wider for better time trend visualization for label, df in data.items(): - # 确保数据已排序 + # Ensure data is sorted df_plot = df.sort_values("bin_sec") - - # 方案 A: 增加线宽和抗锯齿,使用半透明填充让多条线重叠时更容易分辨 + + # Option A: Increase line width and anti-aliasing; use alpha for transparency to distinguish overlaps line, = plt.plot( - df_plot["bin_sec"], - df_plot["freshness"], - label=label, - linewidth=1.8, - alpha=0.9, + df_plot["bin_sec"], + df_plot["freshness"], + label=label, + linewidth=1.8, + alpha=0.9, antialiased=True ) - -# 坐标轴美化 + +# Axis labeling and beautification plt.xlabel("Time (sec)", fontsize=11, fontweight='bold') plt.ylabel(f"Freshness (ms, {BIN_SECONDS}s average)", fontsize=11, fontweight='bold') -# 去掉上方和右侧的边框,让图表更清爽 +# Remove top and right spines for a cleaner look sns.despine() plt.title( - f"Freshness Oscillations\n({BIN_SECONDS}s Binning, Skip {SKIP_SECONDS}s)", - fontsize=13, + f"Freshness Oscillations\n({BIN_SECONDS}s Binning, Skip {SKIP_SECONDS}s)", + fontsize=13, pad=15 ) -# 调整图例到外面或右上角,防止遮挡曲线 +# Move legend outside or to the top right to avoid blocking curves plt.legend(bbox_to_anchor=(1.05, 1), loc='upper left', borderaxespad=0.) plt.grid(True, which="major", ls="-", alpha=0.4) plt.tight_layout() -plt.savefig("freshness_over_time_smooth.png", dpi=300) # 提高分辨率 +plt.savefig("freshness_over_time_smooth.png", dpi=300) # Save with high resolution plt.close() ########################################## -# 图 2:翻转轴后的 CDF(X轴 0-1,步长 0.1) +# Plot 2: Inverted CDF (X-axis 0-1, Step 0.1) ########################################## plt.figure(figsize=(10, 5)) for label, df in data.items(): vals = np.sort(df["freshness"].dropna()) prob = np.linspace(0, 1, len(vals)) - - # x轴为概率 [0, 1],y轴为数值 + + # X-axis is probability [0, 1], Y-axis is value plt.plot(prob, vals, label=label) -# 设置 X 轴刻度:从 0 到 1.1(不包含1.1),步长 0.1 +# Set X-axis ticks: from 0 to 1.1 (excluding 1.1) with step 0.1 plt.xticks(np.arange(0, 1.1, 0.1)) -plt.xlim(0, 1) # 强制显示范围在 0 到 1 之间 +plt.xlim(0, 1) # Force display range between 0 and 1 # plt.yscale("log") plt.xlabel("CDF (Probability)") @@ -142,4 +144,4 @@ plt.savefig("freshness_cdf_fixed_ticks.png") plt.close() -print("图已生成: freshness_over_time_variable_bin.png, freshness_cdf_variable_bin.png") +print("Plots generated: freshness_over_time_smooth.png, freshness_cdf_fixed_ticks.png") \ No newline at end of file diff --git a/perf_rate.py b/perf_rate.py index 55f49fc..797cce5 100644 --- a/perf_rate.py +++ b/perf_rate.py @@ -4,7 +4,7 @@ from datetime import datetime, date ########################################## -# 配置 CSV 文件 和 标签 +# Configuration: CSV Files and Labels ########################################## csv_files = { # "10k": "resulti7i/10k_rate_2.csv", @@ -20,66 +20,66 @@ COL_NAMES = ["time", "rows", "txns", "debezium", "serdRows", "serdTxs"] PLOT_COL = "rows" -MAX_SECONDS = 1800 -SKIP_SECONDS = 10 -BIN_SECONDS = 60 +MAX_SECONDS = 1800 +SKIP_SECONDS = 10 +BIN_SECONDS = 60 ########################################## -# 加载并处理数据 +# Data Loading and Processing ########################################## data = {} for label, path in csv_files.items(): - print(f"正在处理: {label} -> {path}") - - # 1. 加载数据 + print(f"Processing: {label} -> {path}") + + # 1. Load data df = pd.read_csv(path, header=None, names=COL_NAMES, sep=',') - # 2. 【核心修改】处理时间戳并跳过格式不对的行 - # errors='coerce' 会将无法解析的格式转为 NaT + # 2. Handle timestamps and skip rows with incorrect formats + # errors='coerce' turns unparseable formats into NaT df["ts"] = pd.to_datetime(df["time"], format="%H:%M:%S", errors='coerce') - - # 剔除无法解析时间的行 (NaT) + + # Remove rows where time could not be parsed (NaT) initial_count = len(df) df = df.dropna(subset=["ts"]).copy() if len(df) < initial_count: - print(f" 注意: 跳过了 {initial_count - len(df)} 行格式不正确的数据") + print(f" Note: Skipped {initial_count - len(df)} rows with incorrect data format") - # 合并日期 + # Combine with current date df["ts"] = df["ts"].dt.time.apply(lambda x: datetime.combine(date.today(), x)) - # 3. 计算相对时间 - df = df.sort_values("ts") # 确保时间有序 + # 3. Calculate relative time + df = df.sort_values("ts") # Ensure time is ordered t0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t0).dt.total_seconds() - # 4. 过滤时间范围 + # 4. Filter time range df = df[df["sec"] >= SKIP_SECONDS].copy() if df.empty: - print(f" 警告: {label} 在跳过 {SKIP_SECONDS}s 后没有剩余数据") + print(f" Warning: {label} has no data remaining after skipping {SKIP_SECONDS}s") continue t_new0 = df["ts"].iloc[0] df["sec"] = (df["ts"] - t_new0).dt.total_seconds() df = df[df["sec"] <= MAX_SECONDS] - # 5. 【核心修改】重采样聚合 - # 设置索引前,先确保 PLOT_COL 等数值列是 numeric 类型 - # 这样可以防止其他列中混入字符串导致 mean() 失败 + # 5. Resampling and Aggregation + # Ensure numeric columns are numeric types before setting index + # This prevents mean() failures if strings are mixed in numeric columns for col in ["rows", "txns", "debezium", "serdRows", "serdTxs"]: df[col] = pd.to_numeric(df[col], errors='coerce') df = df.set_index("ts") - - # 只对数值列进行 mean 运算,忽略字符串列(如 time 列) + + # Perform mean calculation on numeric columns only, ignoring strings (like 'time' column) df_bin = df.resample(f"{BIN_SECONDS}s").mean(numeric_only=True).reset_index() - # 6. 对齐横轴 + # 6. Align horizontal axis if not df_bin.empty: df_bin["bin_sec"] = (df_bin["ts"] - df_bin["ts"].iloc[0]).dt.total_seconds() data[label] = df_bin ########################################## -# 图 1:时间序列波动 +# Plot 1: Time Series Fluctuations ########################################## plt.figure(figsize=(10, 5)) for label, df in data.items(): @@ -95,7 +95,7 @@ plt.close() ########################################## -# 图 2:CDF +# Plot 2: CDF (Cumulative Distribution Function) ########################################## plt.figure(figsize=(10, 5)) for label, df in data.items(): @@ -113,4 +113,4 @@ plt.savefig(f"rate_{PLOT_COL}_cdf_variable_bin.png") plt.close() -print(f"\n全部完成! 图已生成。") \ No newline at end of file +print(f"\nAll tasks completed! Plots have been generated.") \ No newline at end of file diff --git a/perf_web_monitor.py b/perf_web_monitor.py index 2ae816f..d8f5caf 100644 --- a/perf_web_monitor.py +++ b/perf_web_monitor.py @@ -1,4 +1,4 @@ -from flask import Flask, render_template, jsonify, request +from flask import Flask, render_template, jsonify, request, abort import pandas as pd import os from functools import lru_cache @@ -12,19 +12,22 @@ app = Flask(__name__, template_folder='develop') -# ---- 1. 文件读取缓存(减少频繁IO) ---- +# ---- 1. File Reading Cache (Reduces frequent I/O) ---- @lru_cache(maxsize=64) def _read_csv_cached(path, mtime): """Read CSV with simple caching by modification time.""" df = pd.read_csv(path, names=["time", "rows", "txns", "debezium", "serdRows", "serdTxs"]) + # Take only the last 300 records for real-time display df = df.tail(300) - # drop continuous zero lines + + # Drop rows where all numeric values are zero mask = (df.drop(columns=["time"]).sum(axis=1) != 0) df = df.loc[mask] + return {col: df[col].tolist() for col in df.columns} def read_csv_with_cache(path): - """Wrapper that invalidates cache when file changes.""" + """Wrapper that invalidates cache when file modification time changes.""" try: mtime = os.path.getmtime(path) return _read_csv_cached(path, mtime) @@ -32,14 +35,15 @@ def read_csv_with_cache(path): print(f"[WARN] Failed to read {path}: {e}") return None -# ---- 2. 首页 ---- +# ---- 2. Main Index Page ---- @app.route('/') def index(): return render_template('index.html') -# ---- 3. 文件列表 ---- +# ---- 3. CSV File List ---- @app.route('/list') def list_csv(): + """Returns a list of all CSV files in the data directory.""" try: files = [ f for f in os.listdir(DATA_DIR) @@ -50,25 +54,29 @@ def list_csv(): print(f"[ERROR] list_csv failed: {e}") return jsonify([]) +# ---- 4. Data Retrieval for Specific File ---- @app.route('/data/') def get_data_file(filename): - print(filename) + print(f"Requesting data for: {filename}") path = os.path.join(DATA_DIR, filename) + if not os.path.exists(path): abort(404, description=f"File {filename} not found") data = read_csv_with_cache(path) if not data: - print("Empty") - return jsonify({}) # 文件为空返回空对象 + print("File is empty or could not be processed") + return jsonify({}) # Return empty object if file processing fails return jsonify(data) -# ---- 5. 简单健康检查 ---- +# ---- 5. Simple Health Check ---- @app.route('/health') def health(): return jsonify({"status": "ok", "time": time()}) -# ---- 6. 启动 ---- +# ---- 6. Server Startup ---- if __name__ == '__main__': + # Ensure the data directory exists os.makedirs(DATA_DIR, exist_ok=True) - app.run(host='0.0.0.0', port=PORT, debug=False) + # Run server on all interfaces at the configured port + app.run(host='0.0.0.0', port=PORT, debug=False) \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java index 2480983..f6329d7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkApp.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink; import io.pixelsdb.pixels.sink.config.CommandLineConfig; @@ -41,38 +41,31 @@ /** * Run PixelsSink as a server */ -public class PixelsSinkApp -{ +public class PixelsSinkApp { private static final Logger LOGGER = LoggerFactory.getLogger(PixelsSinkApp.class); private static SinkSource sinkSource; private static HTTPServer prometheusHttpServer; private static FreshnessClient freshnessClient; - public static void main(String[] args) throws IOException - { + public static void main(String[] args) throws IOException { Runtime.getRuntime().addShutdownHook(new Thread(() -> { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); TransactionProxy.staticClose(); sinkSource.stopProcessor(); LOGGER.info("Pixels Sink Server shutdown complete"); - if (config.getSinkMonitorFreshnessLevel().equals("embed") && freshnessClient != null) - { + if (config.getSinkMonitorFreshnessLevel().equals("embed") && freshnessClient != null) { freshnessClient.stop(); } - if (prometheusHttpServer != null) - { + if (prometheusHttpServer != null) { prometheusHttpServer.close(); } MetricsFacade.getInstance().stop(); PixelsSinkWriter pixelsSinkWriter = PixelsSinkWriterFactory.getWriter(); - if(pixelsSinkWriter != null) - { - try - { + if (pixelsSinkWriter != null) { + try { pixelsSinkWriter.close(); - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException(e); } } @@ -83,28 +76,23 @@ public static void main(String[] args) throws IOException PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); sinkSource = SinkSourceFactory.createSinkSource(); - try - { - if (config.isMonitorEnabled()) - { + try { + if (config.isMonitorEnabled()) { DefaultExports.initialize(); prometheusHttpServer = new HTTPServer(config.getMonitorPort()); } - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException(e); } - if (config.getSinkMonitorFreshnessLevel().equals("embed")) - { + if (config.getSinkMonitorFreshnessLevel().equals("embed")) { freshnessClient = FreshnessClient.getInstance(); freshnessClient.start(); } sinkSource.start(); } - private static void init(String[] args) throws IOException - { + private static void init(String[] args) throws IOException { CommandLineConfig cmdLineConfig = new CommandLineConfig(args); PixelsSinkConfigFactory.initialize(cmdLineConfig.getConfigPath()); MetricsFacade metricsFacade = MetricsFacade.getInstance(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java index a72212d..15749c4 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/PixelsSinkProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink; import io.pixelsdb.pixels.common.sink.SinkProvider; @@ -27,12 +27,10 @@ import io.pixelsdb.pixels.sink.source.SinkSourceFactory; import io.pixelsdb.pixels.sink.util.MetricsFacade; -public class PixelsSinkProvider implements SinkProvider -{ +public class PixelsSinkProvider implements SinkProvider { private SinkSource sinkSource; - public void start(ConfigFactory config) - { + public void start(ConfigFactory config) { PixelsSinkConfigFactory.initialize(config); MetricsFacade.getInstance(); sinkSource = SinkSourceFactory.createSinkSource(); @@ -40,14 +38,12 @@ public void start(ConfigFactory config) } @Override - public void shutdown() - { + public void shutdown() { sinkSource.stopProcessor(); } @Override - public boolean isRunning() - { + public boolean isRunning() { return sinkSource.isRunning(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java index d2d2481..4afc3a8 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/CommandLineConfig.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; @@ -27,12 +27,10 @@ import net.sourceforge.argparse4j.inf.Namespace; -public class CommandLineConfig -{ +public class CommandLineConfig { private String configPath; - public CommandLineConfig(String[] args) - { + public CommandLineConfig(String[] args) { ArgumentParser parser = ArgumentParsers .newFor("Pixels-Sink") .build() @@ -44,20 +42,17 @@ public CommandLineConfig(String[] args) .required(false) .help("config path"); - try - { + try { Namespace res = parser.parseArgs(args); this.configPath = res.getString("config"); - } catch (ArgumentParserException e) - { + } catch (ArgumentParserException e) { parser.handleError(e); System.exit(1); } } - public String getConfigPath() - { + public String getConfigPath() { return configPath; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java index 917cad3..1a46ddf 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigKey.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; @@ -34,8 +34,7 @@ */ @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.FIELD) -public @interface ConfigKey -{ +public @interface ConfigKey { String value(); String defaultValue() default ""; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java index c53465d..086638f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/ConfigLoader.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; @@ -29,72 +29,52 @@ import java.util.List; import java.util.Properties; -public class ConfigLoader -{ - public static void load(Properties props, Object target) - { - try - { +public class ConfigLoader { + public static void load(Properties props, Object target) { + try { Class clazz = target.getClass(); - for (Field field : clazz.getDeclaredFields()) - { + for (Field field : clazz.getDeclaredFields()) { ConfigKey annotation = field.getAnnotation(ConfigKey.class); - if (annotation != null) - { + if (annotation != null) { String key = annotation.value(); String value = props.getProperty(key); - if (value == null || value.isEmpty()) - { - if (!annotation.defaultValue().isEmpty()) - { + if (value == null || value.isEmpty()) { + if (!annotation.defaultValue().isEmpty()) { value = annotation.defaultValue(); - } else if (annotation.defaultClass() != Void.class) - { + } else if (annotation.defaultClass() != Void.class) { value = annotation.defaultClass().getName(); } } Object parsed = convert(value, field.getType()); field.setAccessible(true); - try - { + try { field.set(target, parsed); - } catch (IllegalAccessException e) - { + } catch (IllegalAccessException e) { throw new RuntimeException("Failed to inject config for " + key, e); } } } - } catch (Exception e) - { + } catch (Exception e) { throw new RuntimeException("Failed to load config", e); } } - private static Object convert(String value, Class type) - { - if (type.equals(int.class) || type.equals(Integer.class)) - { + private static Object convert(String value, Class type) { + if (type.equals(int.class) || type.equals(Integer.class)) { return Integer.parseInt(value); - } else if (type.equals(long.class) || type.equals(Long.class)) - { + } else if (type.equals(long.class) || type.equals(Long.class)) { return Long.parseLong(value); - } else if (type.equals(short.class) || type.equals(Short.class)) - { + } else if (type.equals(short.class) || type.equals(Short.class)) { return Short.parseShort(value); - } else if (type.equals(boolean.class) || type.equals(Boolean.class)) - { + } else if (type.equals(boolean.class) || type.equals(Boolean.class)) { return Boolean.parseBoolean(value); - } else if (type.equals(PixelsSinkMode.class)) - { + } else if (type.equals(PixelsSinkMode.class)) { return PixelsSinkMode.fromValue(value); - } else if (type.equals(TransactionMode.class)) - { + } else if (type.equals(TransactionMode.class)) { return TransactionMode.fromValue(value); - } else if (type.equals(RetinaServiceProxy.RetinaWriteMode.class)) - { + } else if (type.equals(RetinaServiceProxy.RetinaWriteMode.class)) { return RetinaServiceProxy.RetinaWriteMode.fromValue(value); - } else if (type.equals(List.class)) - { + } else if (type.equals(List.class)) { // Handle List type: split the string by comma (",") // and return a List. Trimming each element is recommended. if (value == null || value.isEmpty()) { diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java index e86ef83..a1f77d2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConfig.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; import io.pixelsdb.pixels.common.utils.ConfigFactory; @@ -32,8 +32,7 @@ import java.util.List; @Getter -public class PixelsSinkConfig -{ +public class PixelsSinkConfig { private final ConfigFactory config; @ConfigKey(value = "transaction.timeout", defaultValue = TransactionConfig.DEFAULT_TRANSACTION_TIME_OUT) @@ -213,26 +212,22 @@ public class PixelsSinkConfig @ConfigKey(value = "trino.parallel", defaultValue = "1") private int trinoParallel; - public PixelsSinkConfig(String configFilePath) throws IOException - { + public PixelsSinkConfig(String configFilePath) throws IOException { this.config = ConfigFactory.Instance(); this.config.loadProperties(configFilePath); init(); } - public PixelsSinkConfig(ConfigFactory config) - { + public PixelsSinkConfig(ConfigFactory config) { this.config = config; init(); } - public String[] getIncludeTables() - { + public String[] getIncludeTables() { return includeTablesRaw.isEmpty() ? new String[0] : includeTablesRaw.split(","); } - private void init() - { + private void init() { ConfigLoader.load(this.config.extractPropertiesByPrefix("", false), this); this.enableSourceRateLimit = this.sourceRateLimit >= 0; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java index 5b6e1f6..3e16fa3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkConstants.java @@ -17,18 +17,16 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; -public final class PixelsSinkConstants -{ +public final class PixelsSinkConstants { public static final String ROW_RECORD_KAFKA_PROP_FACTORY = "row-record"; public static final String TRANSACTION_KAFKA_PROP_FACTORY = "transaction"; public static final int MONITOR_NUM = 2; public static final int MAX_QUEUE_SIZE = 1_000; public static final String SNAPSHOT_TX_PREFIX = "SNAPSHOT-"; - private PixelsSinkConstants() - { + private PixelsSinkConstants() { } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java index 57dec10..2bd04bc 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/PixelsSinkDefaultConfig.java @@ -17,11 +17,10 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; -public class PixelsSinkDefaultConfig -{ +public class PixelsSinkDefaultConfig { public static final String DATA_SOURCE = "kafka"; public static final String PROPERTIES_PATH = "pixels-sink.properties"; public static final String CSV_SINK_PATH = "./data"; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java index cfe1278..cab407a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/TransactionConfig.java @@ -17,13 +17,12 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config; import io.pixelsdb.pixels.sink.event.deserializer.TransactionJsonMessageDeserializer; -public class TransactionConfig -{ +public class TransactionConfig { public static final String DEFAULT_TRANSACTION_TOPIC_SUFFIX = "transaction"; public static final String DEFAULT_TRANSACTION_TOPIC_VALUE_DESERIALIZER = TransactionJsonMessageDeserializer.class.getName(); public static final String DEFAULT_TRANSACTION_TOPIC_GROUP_ID = "transaction_consumer"; diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java index 183851d..405a245 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactory.java @@ -17,14 +17,13 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import java.util.Properties; -public interface KafkaPropFactory -{ +public interface KafkaPropFactory { Properties createKafkaProperties(PixelsSinkConfig config); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java index 50fb4c4..0ec0736 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/KafkaPropFactorySelector.java @@ -17,27 +17,23 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; import java.util.HashMap; -public class KafkaPropFactorySelector -{ +public class KafkaPropFactorySelector { private final HashMap factories = new HashMap<>(); - public KafkaPropFactorySelector() - { + public KafkaPropFactorySelector() { factories.put(PixelsSinkConstants.TRANSACTION_KAFKA_PROP_FACTORY, new TransactionKafkaPropFactory()); factories.put(PixelsSinkConstants.ROW_RECORD_KAFKA_PROP_FACTORY, new RowRecordKafkaPropFactory()); } - public KafkaPropFactory getFactory(String type) - { - if (!factories.containsKey(type)) - { + public KafkaPropFactory getFactory(String type) { + if (!factories.containsKey(type)) { throw new IllegalArgumentException("Unknown factory type: " + type); } return factories.get(type); diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java index 268ae4b..413bc0d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/PixelsSinkConfigFactory.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.common.utils.ConfigFactory; @@ -25,44 +25,36 @@ import java.io.IOException; -public class PixelsSinkConfigFactory -{ +public class PixelsSinkConfigFactory { private static volatile PixelsSinkConfig instance; private static String configFilePath; private static ConfigFactory config; - private PixelsSinkConfigFactory() - { + private PixelsSinkConfigFactory() { } - public static synchronized void initialize(String configFilePath) throws IOException - { - if (instance != null) - { + public static synchronized void initialize(String configFilePath) throws IOException { + if (instance != null) { throw new IllegalStateException("PixelsSinkConfig is already initialized!"); } instance = new PixelsSinkConfig(configFilePath); PixelsSinkConfigFactory.configFilePath = configFilePath; } - public static synchronized void initialize(ConfigFactory config) - { + public static synchronized void initialize(ConfigFactory config) { PixelsSinkConfigFactory.config = config; instance = new PixelsSinkConfig(config); } - public static PixelsSinkConfig getInstance() - { - if (instance == null) - { + public static PixelsSinkConfig getInstance() { + if (instance == null) { throw new IllegalStateException("PixelsSinkConfig is not initialized! Call initialize() first."); } return instance; } - public static synchronized void reset() - { + public static synchronized void reset() { instance = null; configFilePath = null; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java index e6bb211..a31e376 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/RowRecordKafkaPropFactory.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -25,10 +25,8 @@ import java.util.Properties; -public class RowRecordKafkaPropFactory implements KafkaPropFactory -{ - static Properties getCommonKafkaProperties(PixelsSinkConfig config) - { +public class RowRecordKafkaPropFactory implements KafkaPropFactory { + static Properties getCommonKafkaProperties(PixelsSinkConfig config) { Properties kafkaProperties = new Properties(); kafkaProperties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.getBootstrapServers()); kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, config.getKeyDeserializer()); @@ -37,8 +35,7 @@ static Properties getCommonKafkaProperties(PixelsSinkConfig config) } @Override - public Properties createKafkaProperties(PixelsSinkConfig config) - { + public Properties createKafkaProperties(PixelsSinkConfig config) { Properties kafkaProperties = getCommonKafkaProperties(config); kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, config.getValueDeserializer()); kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, config.getGroupId()); diff --git a/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java b/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java index 7b50c46..e97b731 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/config/factory/TransactionKafkaPropFactory.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.config.factory; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -27,11 +27,9 @@ import static io.pixelsdb.pixels.sink.config.factory.RowRecordKafkaPropFactory.getCommonKafkaProperties; -public class TransactionKafkaPropFactory implements KafkaPropFactory -{ +public class TransactionKafkaPropFactory implements KafkaPropFactory { @Override - public Properties createKafkaProperties(PixelsSinkConfig config) - { + public Properties createKafkaProperties(PixelsSinkConfig config) { Properties kafkaProperties = getCommonKafkaProperties(config); kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, config.getTransactionTopicValueDeserializer()); kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, config.getTransactionTopicGroupId() + "-" + config.getGroupId()); diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java index a7afa2a..101dbef 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/RowChangeEvent.java @@ -17,20 +17,16 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event; -import com.google.common.hash.Hashing; import com.google.protobuf.ByteString; import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; -import io.pixelsdb.pixels.common.node.BucketCache; import io.pixelsdb.pixels.common.utils.RetinaUtils; import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.index.IndexProto; import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.metadata.TableMetadata; import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; @@ -38,7 +34,6 @@ import io.prometheus.client.Summary; import lombok.Getter; import lombok.Setter; -import org.apache.logging.log4j.core.util.Assert; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -47,8 +42,7 @@ import java.util.Map; import java.util.stream.IntStream; -public class RowChangeEvent -{ +public class RowChangeEvent { @Getter private final SinkProto.RowRecord rowRecord; @@ -73,7 +67,7 @@ public class RowChangeEvent @Getter private IndexProto.IndexKey afterKey; - private boolean indexKeyInited = false; + private boolean indexKeyInited = false; @Getter private long tableId; @@ -81,8 +75,7 @@ public class RowChangeEvent @Getter private SchemaTableName schemaTableName; - public RowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException - { + public RowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException { this.rowRecord = rowRecord; TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); this.schema = tableMetadataRegistry.getTypeDescription(getSchemaName(), getTable()); @@ -90,8 +83,7 @@ public RowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException initIndexKey(); } - public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) throws SinkException - { + public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) throws SinkException { this.rowRecord = rowRecord; this.schema = schema; @@ -99,8 +91,15 @@ public RowChangeEvent(SinkProto.RowRecord rowRecord, TypeDescription schema) thr // initIndexKey(); } - private void init() throws SinkException - { + protected static int getBucketFromIndexKey(IndexProto.IndexKey indexKey) { + return getBucketIdFromByteBuffer(indexKey.getKey()); + } + + protected static int getBucketIdFromByteBuffer(ByteString byteString) { + return RetinaUtils.getBucketIdFromByteBuffer(byteString); + } + + private void init() throws SinkException { TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); this.tableId = tableMetadataRegistry.getTableId(getSchemaName(), getTable()); this.schemaTableName = new SchemaTableName(getSchemaName(), getTable()); @@ -108,31 +107,25 @@ private void init() throws SinkException initColumnValueMap(); } - private void initColumnValueMap() - { - if (hasBeforeData()) - { + private void initColumnValueMap() { + if (hasBeforeData()) { this.beforeValueMap = new HashMap<>(); initColumnValueMap(rowRecord.getBefore(), beforeValueMap); } - if (hasAfterData()) - { + if (hasAfterData()) { this.afterValueMap = new HashMap<>(); initColumnValueMap(rowRecord.getAfter(), afterValueMap); } } - private void initColumnValueMap(SinkProto.RowValue rowValue, Map map) - { + private void initColumnValueMap(SinkProto.RowValue rowValue, Map map) { IntStream.range(0, schema.getFieldNames().size()) .forEach(i -> map.put(schema.getFieldNames().get(i), rowValue.getValuesList().get(i))); } - public void initIndexKey() throws SinkException - { - if(indexKeyInited) - { + public void initIndexKey() throws SinkException { + if (indexKeyInited) { return; } @@ -140,55 +133,44 @@ public void initIndexKey() throws SinkException this.rowRecord.getSource().getDb(), this.rowRecord.getSource().getTable()); - if (!this.tableMetadata.hasPrimaryIndex()) - { + if (!this.tableMetadata.hasPrimaryIndex()) { return; } - if (hasBeforeData()) - { + if (hasBeforeData()) { this.beforeKey = generateIndexKey(tableMetadata, beforeValueMap); } - if (hasAfterData()) - { + if (hasAfterData()) { this.afterKey = generateIndexKey(tableMetadata, afterValueMap); } indexKeyInited = true; } - public void updateIndexKey() throws SinkException - { - if (hasBeforeData()) - { + public void updateIndexKey() throws SinkException { + if (hasBeforeData()) { this.beforeKey = generateIndexKey(tableMetadata, beforeValueMap); } - if (hasAfterData()) - { + if (hasAfterData()) { this.afterKey = generateIndexKey(tableMetadata, afterValueMap); } } - public int getBeforeBucketFromIndex() - { + public int getBeforeBucketFromIndex() { assert indexKeyInited; - if(hasBeforeData()) - { + if (hasBeforeData()) { return getBucketFromIndexKey(beforeKey); } throw new IllegalCallerException("Event dosen't have before data"); } - public boolean isPkChanged() throws SinkException - { - if(!indexKeyInited) - { + public boolean isPkChanged() throws SinkException { + if (!indexKeyInited) { initIndexKey(); } - if(getOp() != SinkProto.OperationType.UPDATE) - { + if (getOp() != SinkProto.OperationType.UPDATE) { return false; } @@ -198,44 +180,28 @@ public boolean isPkChanged() throws SinkException return !beforeKey.equals(afterKey); } - public int getAfterBucketFromIndex() - { + public int getAfterBucketFromIndex() { assert indexKeyInited; - if(hasAfterData()) - { + if (hasAfterData()) { return getBucketFromIndexKey(afterKey); } throw new IllegalCallerException("Event dosen't have after data"); } - protected static int getBucketFromIndexKey(IndexProto.IndexKey indexKey) - { - return getBucketIdFromByteBuffer(indexKey.getKey()); - } - - protected static int getBucketIdFromByteBuffer(ByteString byteString) - { - return RetinaUtils.getBucketIdFromByteBuffer(byteString); - } - - - private IndexProto.IndexKey generateIndexKey(TableMetadata tableMetadata, Map rowValue) - { + private IndexProto.IndexKey generateIndexKey(TableMetadata tableMetadata, Map rowValue) { List keyColumnNames = tableMetadata.getKeyColumnNames(); SinglePointIndex index = tableMetadata.getIndex(); int len = keyColumnNames.size(); List keyColumnValues = new ArrayList<>(len); int keySize = 0; - for (String keyColumnName : keyColumnNames) - { + for (String keyColumnName : keyColumnNames) { ByteString value = rowValue.get(keyColumnName).getValue(); keyColumnValues.add(value); keySize += value.size(); } ByteBuffer byteBuffer = ByteBuffer.allocate(keySize); - for (ByteString value : keyColumnValues) - { + for (ByteString value : keyColumnValues) { byteBuffer.put(value.toByteArray()); } @@ -247,117 +213,96 @@ private IndexProto.IndexKey generateIndexKey(TableMetadata tableMetadata, Map getAfterData() - { + public List getAfterData() { List colValues = rowRecord.getAfter().getValuesList(); List colValueList = new ArrayList<>(colValues.size()); - for (SinkProto.ColumnValue col : colValues) - { + for (SinkProto.ColumnValue col : colValues) { colValueList.add(col.getValue()); } return colValueList; } @Override - public String toString() - { + public String toString() { String sb = "RowChangeEvent{" + rowRecord.getSource().getDb() + "." + rowRecord.getSource().getTable() + diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java index d47242a..03fe0a7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/DeserializerUtil.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.sink.SinkProto; @@ -25,67 +25,52 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; -public class DeserializerUtil -{ - static public SinkProto.TransactionStatus getStatusSafely(T record, String field) - { +public class DeserializerUtil { + static public SinkProto.TransactionStatus getStatusSafely(T record, String field) { String statusString = getStringSafely(record, field); - if (statusString.equals("BEGIN")) - { + if (statusString.equals("BEGIN")) { return SinkProto.TransactionStatus.BEGIN; } - if (statusString.equals("END")) - { + if (statusString.equals("END")) { return SinkProto.TransactionStatus.END; } return SinkProto.TransactionStatus.UNRECOGNIZED; } - public static Object getFieldSafely(T record, String field) - { - try - { - if (record instanceof GenericRecord avro) - { + public static Object getFieldSafely(T record, String field) { + try { + if (record instanceof GenericRecord avro) { return avro.get(field); - } else if (record instanceof Struct struct) - { + } else if (record instanceof Struct struct) { return struct.get(field); - } else if (record instanceof SourceRecord sourceRecord) - { + } else if (record instanceof SourceRecord sourceRecord) { return ((Struct) sourceRecord.value()).get(field); } - } catch (Exception e) - { + } catch (Exception e) { return null; } return null; } - public static String getStringSafely(T record, String field) - { + public static String getStringSafely(T record, String field) { Object value = getFieldSafely(record, field); return value != null ? value.toString() : ""; } - public static Long getLongSafely(T record, String field) - { + public static Long getLongSafely(T record, String field) { Object value = getFieldSafely(record, field); return value instanceof Number ? ((Number) value).longValue() : 0L; } - public static Integer getIntSafely(T record, String field) - { + public static Integer getIntSafely(T record, String field) { Object value = getFieldSafely(record, field); return value instanceof Number ? ((Number) value).intValue() : 0; } - static public SinkProto.OperationType getOperationType(String op) - { + static public SinkProto.OperationType getOperationType(String op) { op = op.toLowerCase(); - return switch (op) - { + return switch (op) { case "c" -> SinkProto.OperationType.INSERT; case "u" -> SinkProto.OperationType.UPDATE; case "d" -> SinkProto.OperationType.DELETE; @@ -94,18 +79,15 @@ static public SinkProto.OperationType getOperationType(String op) }; } - static public boolean hasBeforeValue(SinkProto.OperationType op) - { + static public boolean hasBeforeValue(SinkProto.OperationType op) { return op == SinkProto.OperationType.DELETE || op == SinkProto.OperationType.UPDATE; } - static public boolean hasAfterValue(SinkProto.OperationType op) - { + static public boolean hasAfterValue(SinkProto.OperationType op) { return op != SinkProto.OperationType.DELETE; } - static public String getTransIdPrefix(String originTransID) - { + static public String getTransIdPrefix(String originTransID) { return originTransID.contains(":") ? originTransID.substring(0, originTransID.indexOf(":")) : originTransID; diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java index a451235..81d9036 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventAvroDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import io.apicurio.registry.serde.SerdeConfig; @@ -37,16 +37,14 @@ import java.util.HashMap; import java.util.Map; -public class RowChangeEventAvroDeserializer implements Deserializer -{ +public class RowChangeEventAvroDeserializer implements Deserializer { private final AvroKafkaDeserializer avroDeserializer = new AvroKafkaDeserializer<>(); private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); private final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); @Override - public void configure(Map configs, boolean isKey) - { + public void configure(Map configs, boolean isKey) { Map enrichedConfig = new HashMap<>(configs); enrichedConfig.put(SerdeConfig.REGISTRY_URL, config.getRegistryUrl()); enrichedConfig.put(SerdeConfig.CHECK_PERIOD_MS, SerdeConfig.CHECK_PERIOD_MS_DEFAULT); @@ -54,17 +52,14 @@ public void configure(Map configs, boolean isKey) } @Override - public RowChangeEvent deserialize(String topic, byte[] data) - { - try - { + public RowChangeEvent deserialize(String topic, byte[] data) { + try { MetricsFacade.getInstance().addRawData(data.length); GenericRecord avroRecord = avroDeserializer.deserialize(topic, data); Schema avroSchema = avroRecord.getSchema(); RowChangeEvent rowChangeEvent = convertToRowChangeEvent(avroRecord, avroSchema); return rowChangeEvent; - } catch (Exception e) - { + } catch (Exception e) { e.printStackTrace(); return null; // throw new SerializationException("Avro deserialization failed", e); @@ -72,20 +67,17 @@ public RowChangeEvent deserialize(String topic, byte[] data) } } - private void registerSchema(String topic, Schema avroSchema) - { + private void registerSchema(String topic, Schema avroSchema) { } - private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema schema) throws SinkException - { + private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema schema) throws SinkException { SinkProto.OperationType op = parseOperationType(avroRecord); SinkProto.RowRecord.Builder recordBuilder = SinkProto.RowRecord.newBuilder() .setOp(op) // .setTsMs(DeserializerUtil.getLongSafely(avroRecord, "ts_ms")); ; - if (avroRecord.get("source") != null) - { + if (avroRecord.get("source") != null) { //TODO: 这里看下怎么处理,如果没有source信息,其实可以通过topic推出schema和table信息。 parseSourceInfo((GenericRecord) avroRecord.get("source"), recordBuilder.getSourceBuilder()); } @@ -93,11 +85,9 @@ private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema String sourceSchema = recordBuilder.getSource().getDb(); String sourceTable = recordBuilder.getSource().getTable(); TypeDescription typeDescription = null; - try - { + try { typeDescription = tableMetadataRegistry.getTypeDescription(sourceSchema, sourceTable); - } catch (SinkException e) - { + } catch (SinkException e) { throw new RuntimeException(e); } // TableMetadata tableMetadata = tableMetadataRegistry.loadTableMetadata(sourceSchema, sourceTable); @@ -105,8 +95,7 @@ private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema recordBuilder.setBefore(parseRowData(avroRecord.get("before"), typeDescription)); recordBuilder.setAfter(parseRowData(avroRecord.get("after"), typeDescription)); - if (avroRecord.get("transaction") != null) - { + if (avroRecord.get("transaction") != null) { parseTransactionInfo((GenericRecord) avroRecord.get("transaction"), recordBuilder.getTransactionBuilder()); } @@ -114,31 +103,25 @@ private RowChangeEvent convertToRowChangeEvent(GenericRecord avroRecord, Schema return new RowChangeEvent(recordBuilder.build(), typeDescription); } - private SinkProto.OperationType parseOperationType(GenericRecord record) - { + private SinkProto.OperationType parseOperationType(GenericRecord record) { String op = DeserializerUtil.getStringSafely(record, "op"); - try - { + try { return DeserializerUtil.getOperationType(op); - } catch (IllegalArgumentException e) - { + } catch (IllegalArgumentException e) { return SinkProto.OperationType.UNRECOGNIZED; } } - private SinkProto.RowValue.Builder parseRowData(Object data, TypeDescription typeDescription) - { + private SinkProto.RowValue.Builder parseRowData(Object data, TypeDescription typeDescription) { SinkProto.RowValue.Builder builder = SinkProto.RowValue.newBuilder(); - if (data instanceof GenericRecord rowData) - { + if (data instanceof GenericRecord rowData) { RowDataParser rowDataParser = new RowDataParser(typeDescription); // TODO make it static? rowDataParser.parse(rowData, builder); } return builder; } - private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder builder) - { + private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder builder) { builder .setDb(DeserializerUtil.getStringSafely(source, "db")) @@ -159,8 +142,7 @@ private void parseSourceInfo(GenericRecord source, SinkProto.SourceInfo.Builder } private void parseTransactionInfo(GenericRecord transaction, - SinkProto.TransactionInfo.Builder builder) - { + SinkProto.TransactionInfo.Builder builder) { builder.setId(DeserializerUtil.getTransIdPrefix(DeserializerUtil.getStringSafely(transaction, "id"))) .setTotalOrder(DeserializerUtil.getLongSafely(transaction, "total_order")) .setDataCollectionOrder(DeserializerUtil.getLongSafely(transaction, "data_collection_order")); diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java index 3a850d1..99c8677 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventJsonDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; @@ -33,47 +33,39 @@ import org.slf4j.LoggerFactory; -public class RowChangeEventJsonDeserializer implements Deserializer -{ +public class RowChangeEventJsonDeserializer implements Deserializer { private static final Logger logger = LoggerFactory.getLogger(RowChangeEventJsonDeserializer.class); private static final ObjectMapper objectMapper = new ObjectMapper(); private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); @Override - public RowChangeEvent deserialize(String topic, byte[] data) - { - if (data == null || data.length == 0) - { + public RowChangeEvent deserialize(String topic, byte[] data) { + if (data == null || data.length == 0) { logger.debug("Received empty message from topic: {}", topic); return null; } MetricsFacade.getInstance().addRawData(data.length); - try - { + try { JsonNode rootNode = objectMapper.readTree(data); JsonNode payloadNode = rootNode.path("payload"); SinkProto.OperationType opType = parseOperationType(payloadNode); return buildRowRecord(payloadNode, opType); - } catch (Exception e) - { + } catch (Exception e) { logger.error("Failed to deserialize message from topic {}: {}", topic, e.getMessage()); return null; } } - private SinkProto.OperationType parseOperationType(JsonNode payloadNode) - { + private SinkProto.OperationType parseOperationType(JsonNode payloadNode) { String opCode = payloadNode.path("op").asText(""); return DeserializerUtil.getOperationType(opCode); } @Deprecated - private TypeDescription getSchema(JsonNode schemaNode, SinkProto.OperationType opType) - { - return switch (opType) - { + private TypeDescription getSchema(JsonNode schemaNode, SinkProto.OperationType opType) { + return switch (opType) { case DELETE -> SchemaDeserializer.parseFromBeforeOrAfter(schemaNode, "before"); case INSERT, UPDATE, SNAPSHOT -> SchemaDeserializer.parseFromBeforeOrAfter(schemaNode, "after"); case UNRECOGNIZED -> throw new IllegalArgumentException("Operation type is unknown. Check op"); @@ -81,8 +73,7 @@ private TypeDescription getSchema(JsonNode schemaNode, SinkProto.OperationType o } private RowChangeEvent buildRowRecord(JsonNode payloadNode, - SinkProto.OperationType opType) throws SinkException - { + SinkProto.OperationType opType) throws SinkException { SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); @@ -93,33 +84,28 @@ private RowChangeEvent buildRowRecord(JsonNode payloadNode, String schemaName; String tableName; - if (payloadNode.has("source")) - { + if (payloadNode.has("source")) { SinkProto.SourceInfo.Builder sourceInfoBuilder = parseSourceInfo(payloadNode.get("source")); schemaName = sourceInfoBuilder.getDb(); // Notice we use the schema tableName = sourceInfoBuilder.getTable(); builder.setSource(sourceInfoBuilder); - } else - { + } else { throw new IllegalArgumentException("Missing source field in row record"); } TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); RowDataParser rowDataParser = new RowDataParser(typeDescription); - if (payloadNode.hasNonNull("transaction")) - { + if (payloadNode.hasNonNull("transaction")) { builder.setTransaction(parseTransactionInfo(payloadNode.get("transaction"))); } - if (DeserializerUtil.hasBeforeValue(opType)) - { + if (DeserializerUtil.hasBeforeValue(opType)) { SinkProto.RowValue.Builder beforeBuilder = builder.getBeforeBuilder(); rowDataParser.parse(payloadNode.get("before"), beforeBuilder); builder.setBefore(beforeBuilder); } - if (DeserializerUtil.hasAfterValue(opType)) - { + if (DeserializerUtil.hasAfterValue(opType)) { SinkProto.RowValue.Builder afterBuilder = builder.getAfterBuilder(); rowDataParser.parse(payloadNode.get("after"), afterBuilder); @@ -127,19 +113,16 @@ private RowChangeEvent buildRowRecord(JsonNode payloadNode, } RowChangeEvent event = new RowChangeEvent(builder.build(), typeDescription); - try - { + try { event.initIndexKey(); - } catch (SinkException e) - { + } catch (SinkException e) { logger.warn("Row change event {}: Init index key failed", event); } return event; } - private SinkProto.SourceInfo.Builder parseSourceInfo(JsonNode sourceNode) - { + private SinkProto.SourceInfo.Builder parseSourceInfo(JsonNode sourceNode) { return SinkProto.SourceInfo.newBuilder() .setDb(sourceNode.path("db").asText()) .setSchema(sourceNode.path("schema").asText()) @@ -160,8 +143,7 @@ private SinkProto.SourceInfo.Builder parseSourceInfo(JsonNode sourceNode) ; } - private SinkProto.TransactionInfo parseTransactionInfo(JsonNode txNode) - { + private SinkProto.TransactionInfo parseTransactionInfo(JsonNode txNode) { return SinkProto.TransactionInfo.newBuilder() .setId(DeserializerUtil.getTransIdPrefix(txNode.path("id").asText())) .setTotalOrder(txNode.path("total_order").asLong()) diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java index 5154bd1..7cd4ba1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowChangeEventStructDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; @@ -38,21 +38,18 @@ * @author: AntiO2 * @date: 2025/9/26 12:00 */ -public class RowChangeEventStructDeserializer -{ +public class RowChangeEventStructDeserializer { private static final Logger LOGGER = Logger.getLogger(RowChangeEventStructDeserializer.class.getName()); private static final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); - public static RowChangeEvent convertToRowChangeEvent(SourceRecord sourceRecord) throws SinkException - { + public static RowChangeEvent convertToRowChangeEvent(SourceRecord sourceRecord) throws SinkException { Struct value = (Struct) sourceRecord.value(); String op = value.getString("op"); SinkProto.OperationType operationType = DeserializerUtil.getOperationType(op); return buildRowRecord(value, operationType); } - public static RowChangeEvent convertToRowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException - { + public static RowChangeEvent convertToRowChangeEvent(SinkProto.RowRecord rowRecord) throws SinkException { String schemaName = rowRecord.getSource().getDb(); String tableName = rowRecord.getSource().getTable(); TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); @@ -60,8 +57,7 @@ public static RowChangeEvent convertToRowChangeEvent(SinkProto.RowRecord rowReco } private static RowChangeEvent buildRowRecord(Struct value, - SinkProto.OperationType opType) throws SinkException - { + SinkProto.OperationType opType) throws SinkException { SinkProto.RowRecord.Builder builder = SinkProto.RowRecord.newBuilder(); @@ -69,15 +65,13 @@ private static RowChangeEvent buildRowRecord(Struct value, String schemaName; String tableName; - try - { + try { Struct source = value.getStruct("source"); SinkProto.SourceInfo.Builder sourceInfoBuilder = parseSourceInfo(source); schemaName = sourceInfoBuilder.getDb(); // Notice we use the schema tableName = sourceInfoBuilder.getTable(); builder.setSource(sourceInfoBuilder); - } catch (DataException e) - { + } catch (DataException e) { LOGGER.warning("Missing source field in row record"); throw new SinkException(e); } @@ -85,25 +79,21 @@ private static RowChangeEvent buildRowRecord(Struct value, TypeDescription typeDescription = tableMetadataRegistry.getTypeDescription(schemaName, tableName); RowDataParser rowDataParser = new RowDataParser(typeDescription); - try - { + try { Struct transaction = value.getStruct("transaction"); SinkProto.TransactionInfo transactionInfo = parseTransactionInfo(transaction); builder.setTransaction(transactionInfo); - } catch (DataException e) - { + } catch (DataException e) { LOGGER.warning("Missing transaction field in row record"); } - if (DeserializerUtil.hasBeforeValue(opType)) - { + if (DeserializerUtil.hasBeforeValue(opType)) { SinkProto.RowValue.Builder beforeBuilder = builder.getBeforeBuilder(); rowDataParser.parse(value.getStruct("before"), beforeBuilder); builder.setBefore(beforeBuilder); } - if (DeserializerUtil.hasAfterValue(opType)) - { + if (DeserializerUtil.hasAfterValue(opType)) { SinkProto.RowValue.Builder afterBuilder = builder.getAfterBuilder(); rowDataParser.parse(value.getStruct("after"), afterBuilder); @@ -114,8 +104,7 @@ private static RowChangeEvent buildRowRecord(Struct value, return event; } - private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) - { + private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) { return SinkProto.SourceInfo.newBuilder() // .setVersion(DeserializerUtil.getStringSafely(source, "version")) // .setConnector(DeserializerUtil.getStringSafely(source, "connector")) @@ -133,8 +122,7 @@ private static SinkProto.SourceInfo.Builder parseSourceInfo(T source) // .setXmin(DeserializerUtil.getLongSafely(source, "xmin")); } - private static SinkProto.TransactionInfo parseTransactionInfo(T txNode) - { + private static SinkProto.TransactionInfo parseTransactionInfo(T txNode) { return SinkProto.TransactionInfo.newBuilder() .setId(DeserializerUtil.getTransIdPrefix( DeserializerUtil.getStringSafely(txNode, "id"))) diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java index bce8871..63b7fa2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/RowDataParser.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; @@ -39,30 +39,24 @@ import java.util.Collections; import java.util.Map; -class RowDataParser -{ +class RowDataParser { private final TypeDescription schema; - public RowDataParser(TypeDescription schema) - { + public RowDataParser(TypeDescription schema) { this.schema = schema; } - public void parse(GenericRecord record, SinkProto.RowValue.Builder builder) - { - for (int i = 0; i < schema.getFieldNames().size(); i++) - { + public void parse(GenericRecord record, SinkProto.RowValue.Builder builder) { + for (int i = 0; i < schema.getFieldNames().size(); i++) { String fieldName = schema.getFieldNames().get(i); TypeDescription fieldType = schema.getChildren().get(i); builder.addValues(parseValue(record, fieldName, fieldType).build()); } } - public void parse(JsonNode node, SinkProto.RowValue.Builder builder) - { - for (int i = 0; i < schema.getFieldNames().size(); i++) - { + public void parse(JsonNode node, SinkProto.RowValue.Builder builder) { + for (int i = 0; i < schema.getFieldNames().size(); i++) { String fieldName = schema.getFieldNames().get(i); TypeDescription fieldType = schema.getChildren().get(i); builder.addValues(parseValue(node.get(fieldName), fieldName, fieldType).build()); @@ -70,10 +64,8 @@ public void parse(JsonNode node, SinkProto.RowValue.Builder builder) } - public void parse(Struct record, SinkProto.RowValue.Builder builder) - { - for (int i = 0; i < schema.getFieldNames().size(); i++) - { + public void parse(Struct record, SinkProto.RowValue.Builder builder) { + for (int i = 0; i < schema.getFieldNames().size(); i++) { String fieldName = schema.getFieldNames().get(i); Field field = record.schema().field(fieldName); Schema.Type fieldType = field.schema().type(); @@ -81,10 +73,8 @@ public void parse(Struct record, SinkProto.RowValue.Builder builder) } } - private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fieldName, TypeDescription type) - { - if (valueNode == null || valueNode.isNull()) - { + private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fieldName, TypeDescription type) { + if (valueNode == null || valueNode.isNull()) { return SinkProto.ColumnValue.newBuilder() // .setName(fieldName) .setValue(ByteString.EMPTY); @@ -92,26 +82,22 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); - switch (type.getCategory()) - { - case INT: - { + switch (type.getCategory()) { + case INT: { int value = valueNode.asInt(); byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } - case LONG: - { + case LONG: { long value = valueNode.asLong(); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } - case CHAR: - { + case CHAR: { String text = valueNode.asText(); byte[] bytes = new byte[]{(byte) text.charAt(0)}; columnValueBuilder.setValue(ByteString.copyFrom(bytes)); @@ -121,15 +107,13 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel } case VARCHAR: case STRING: - case VARBINARY: - { + case VARBINARY: { String value = valueNode.asText().trim(); columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } - case DECIMAL: - { + case DECIMAL: { String value = parseDecimal(valueNode, type).toString(); columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder() @@ -138,20 +122,17 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel // .setScale(type.getScale())); break; } - case BINARY: - { + case BINARY: { String base64 = valueNode.asText(); // assume already base64 encoded columnValueBuilder.setValue(ByteString.copyFrom(base64, StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BINARY)); break; } - case STRUCT: - { + case STRUCT: { // You can recursively parse fields in a struct here throw new UnsupportedOperationException("STRUCT parsing not yet implemented"); } - case DOUBLE: - { + case DOUBLE: { double value = valueNode.asDouble(); long longBits = Double.doubleToLongBits(value); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(longBits).array(); @@ -159,8 +140,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); break; } - case FLOAT: - { + case FLOAT: { float value = (float) valueNode.asDouble(); int intBits = Float.floatToIntBits(value); byte[] bytes = ByteBuffer.allocate(4).putInt(intBits).array(); @@ -168,8 +148,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.FLOAT)); break; } - case DATE: - { + case DATE: { int isoDate = valueNode.asInt(); byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(isoDate).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); @@ -177,8 +156,7 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel // .setKind(PixelsProto.Type.Kind.DATE)); break; } - case TIMESTAMP: - { + case TIMESTAMP: { long timestamp = valueNode.asLong(); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(timestamp).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); @@ -195,46 +173,39 @@ private SinkProto.ColumnValue.Builder parseValue(JsonNode valueNode, String fiel @Deprecated // TODO: use bit - private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fieldName, TypeDescription fieldType) - { + private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fieldName, TypeDescription fieldType) { SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); // columnValueBuilder.setName(fieldName); Object raw = record.get(fieldName); - if (raw == null) - { + if (raw == null) { columnValueBuilder.setValue(ByteString.EMPTY); return columnValueBuilder; } - switch (fieldType.getCategory()) - { - case INT: - { + switch (fieldType.getCategory()) { + case INT: { int value = (int) raw; columnValueBuilder.setValue(ByteString.copyFrom(Integer.toString(value), StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } - case LONG: - { + case LONG: { long value = (long) raw; columnValueBuilder.setValue(ByteString.copyFrom(Long.toString(value), StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } - case STRING: - { + case STRING: { String value = raw.toString(); columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } - case DECIMAL: - { + case DECIMAL: { ByteBuffer buffer = (ByteBuffer) raw; String decimalStr = new String(buffer.array(), StandardCharsets.UTF_8).trim(); columnValueBuilder.setValue(ByteString.copyFrom(decimalStr, StandardCharsets.UTF_8)); @@ -245,8 +216,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi break; } - case DATE: - { + case DATE: { int epochDay = (int) raw; String isoDate = LocalDate.ofEpochDay(epochDay).toString(); // e.g., "2025-07-03" columnValueBuilder.setValue(ByteString.copyFrom(isoDate, StandardCharsets.UTF_8)); @@ -254,8 +224,7 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi break; } - case BINARY: - { + case BINARY: { ByteBuffer buffer = (ByteBuffer) raw; // encode as hex or base64 if needed, otherwise just dump as UTF-8 string if it's meant to be readable String base64 = Base64.getEncoder().encodeToString(buffer.array()); @@ -270,59 +239,50 @@ private SinkProto.ColumnValue.Builder parseValue(GenericRecord record, String fi return columnValueBuilder; } - private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName, Schema.Type type) - { + private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName, Schema.Type type) { // TODO(AntiO2) support pixels type - if (record == null) - { + if (record == null) { return SinkProto.ColumnValue.newBuilder() // .setName(fieldName) .setValue(ByteString.EMPTY); } SinkProto.ColumnValue.Builder columnValueBuilder = SinkProto.ColumnValue.newBuilder(); - switch (type) - { + switch (type) { case INT8: case INT16: - case INT32: - { + case INT32: { int value = (Integer) record; byte[] bytes = ByteBuffer.allocate(Integer.BYTES).putInt(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.INT)); break; } - case INT64: - { + case INT64: { long value = (Long) record; byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); columnValueBuilder.setValue(ByteString.copyFrom(bytes)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.LONG)); break; } - case BYTES: - { + case BYTES: { byte[] bytes = (byte[]) record; columnValueBuilder.setValue(ByteString.copyFrom(bytes)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.BYTE)); break; } case BOOLEAN: - case STRING: - { + case STRING: { String value = (String) record; columnValueBuilder.setValue(ByteString.copyFrom(value, StandardCharsets.UTF_8)); // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.STRING)); break; } - case STRUCT: - { + case STRUCT: { // You can recursively parse fields in a struct here throw new UnsupportedOperationException("STRUCT parsing not yet implemented"); } - case FLOAT64: - { + case FLOAT64: { double value = (double) record; long doubleBits = Double.doubleToLongBits(value); byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(doubleBits).array(); @@ -330,8 +290,7 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName // columnValueBuilder.setType(PixelsProto.Type.newBuilder().setKind(PixelsProto.Type.Kind.DOUBLE)); break; } - case FLOAT32: - { + case FLOAT32: { float value = (float) record; int intBits = Float.floatToIntBits(value); byte[] bytes = ByteBuffer.allocate(4).putInt(intBits).array(); @@ -346,30 +305,24 @@ private SinkProto.ColumnValue.Builder parseValue(Object record, String fieldName return columnValueBuilder; } - private Map parseDeleteRecord() - { + private Map parseDeleteRecord() { return Collections.singletonMap("__deleted", true); } - BigDecimal parseDecimal(JsonNode node, TypeDescription type) - { + BigDecimal parseDecimal(JsonNode node, TypeDescription type) { byte[] bytes = Base64.getDecoder().decode(node.asText()); int scale = type.getScale(); return new BigDecimal(new BigInteger(bytes), scale); } - private LocalDate parseDate(JsonNode node) - { + private LocalDate parseDate(JsonNode node) { return LocalDate.ofEpochDay(node.asLong()); } - private byte[] parseBinary(JsonNode node) - { - try - { + private byte[] parseBinary(JsonNode node) { + try { return node.binaryValue(); - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException("Binary parsing failed", e); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java index 02be8ca..3bf3959 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/SchemaDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.JsonNode; @@ -28,34 +28,27 @@ import java.util.Iterator; import java.util.Map; -public class SchemaDeserializer -{ - public static TypeDescription parseFromBeforeOrAfter(JsonNode schemaNode, String fieldName) - { +public class SchemaDeserializer { + public static TypeDescription parseFromBeforeOrAfter(JsonNode schemaNode, String fieldName) { JsonNode beforeAfterSchema = findSchemaField(schemaNode, fieldName); - if (beforeAfterSchema == null) - { + if (beforeAfterSchema == null) { throw new IllegalArgumentException("Field '" + fieldName + "' not found in schema"); } return parseStruct(beforeAfterSchema.get("fields")); } - private static JsonNode findSchemaField(JsonNode schemaNode, String targetField) - { + private static JsonNode findSchemaField(JsonNode schemaNode, String targetField) { Iterator fields = schemaNode.get("fields").elements(); - while (fields.hasNext()) - { + while (fields.hasNext()) { JsonNode field = fields.next(); - if (targetField.equals(field.get("field").asText())) - { + if (targetField.equals(field.get("field").asText())) { return field; } } return null; } - public static TypeDescription parseStruct(JsonNode fields) - { + public static TypeDescription parseStruct(JsonNode fields) { TypeDescription structType = TypeDescription.createStruct(); fields.forEach(field -> { @@ -66,19 +59,15 @@ public static TypeDescription parseStruct(JsonNode fields) return structType; } - static TypeDescription parseFieldType(JsonNode fieldNode) - { - if (!fieldNode.has("type")) - { + static TypeDescription parseFieldType(JsonNode fieldNode) { + if (!fieldNode.has("type")) { throw new IllegalArgumentException("Field is missing required 'type' property"); } String typeName = fieldNode.get("type").asText(); String logicalType = fieldNode.has("name") ? fieldNode.get("name").asText() : null; - if (logicalType != null) - { - switch (logicalType) - { + if (logicalType != null) { + switch (logicalType) { case "org.apache.kafka.connect.data.Decimal": int precision = Integer.parseInt(fieldNode.get("parameters").get("connect.decimal.precision").asText()); int scale = Integer.parseInt(fieldNode.get("parameters").get("scale").asText()); @@ -88,8 +77,7 @@ static TypeDescription parseFieldType(JsonNode fieldNode) } } - switch (typeName) - { + switch (typeName) { case "int64": return TypeDescription.createLong(); case "int32": @@ -103,11 +91,9 @@ static TypeDescription parseFieldType(JsonNode fieldNode) } } - public static TypeDescription parseFromBeforeOrAfter(Schema schemaNode, String fieldName) - { + public static TypeDescription parseFromBeforeOrAfter(Schema schemaNode, String fieldName) { Schema.Field filed = schemaNode.getField(fieldName); - if (filed == null) - { + if (filed == null) { throw new IllegalArgumentException("Can't find field in avro schema: " + fieldName); } @@ -116,22 +102,18 @@ public static TypeDescription parseFromBeforeOrAfter(Schema schemaNode, String f } - public static TypeDescription parseFromAvroSchema(Schema avroSchema) - { + public static TypeDescription parseFromAvroSchema(Schema avroSchema) { return parseAvroType(avroSchema, new HashMap<>()); } - private static TypeDescription parseAvroType(Schema schema, Map cache) - { + private static TypeDescription parseAvroType(Schema schema, Map cache) { String schemaKey = schema.getFullName() + ":" + schema.hashCode(); - if (cache.containsKey(schemaKey)) - { + if (cache.containsKey(schemaKey)) { return cache.get(schemaKey); } TypeDescription typeDesc; - switch (schema.getType()) - { + switch (schema.getType()) { case RECORD: typeDesc = parseAvroRecord(schema, cache); break; @@ -152,48 +134,38 @@ private static TypeDescription parseAvroType(Schema schema, Map cache) - { + private static TypeDescription parseAvroRecord(Schema schema, Map cache) { TypeDescription structType = TypeDescription.createStruct(); - for (Schema.Field field : schema.getFields()) - { + for (Schema.Field field : schema.getFields()) { TypeDescription fieldType = parseAvroType(field.schema(), cache); structType.addField(field.name(), fieldType); } return structType; } - private static TypeDescription parseAvroUnion(Schema schema, Map cache) - { - for (Schema type : schema.getTypes()) - { - if (type.getType() != Schema.Type.NULL) - { + private static TypeDescription parseAvroUnion(Schema schema, Map cache) { + for (Schema type : schema.getTypes()) { + if (type.getType() != Schema.Type.NULL) { return parseAvroType(type, cache); } } throw new IllegalArgumentException("Invalid union type: " + schema); } - private static TypeDescription parseAvroArray(Schema schema, Map cache) - { + private static TypeDescription parseAvroArray(Schema schema, Map cache) { throw new RuntimeException("Doesn't support Array"); } - private static TypeDescription parseAvroMap(Schema schema, Map cache) - { + private static TypeDescription parseAvroMap(Schema schema, Map cache) { throw new RuntimeException("Doesn't support Map"); } - private static TypeDescription parseAvroPrimitive(Schema schema) - { + private static TypeDescription parseAvroPrimitive(Schema schema) { String logicalType = schema.getLogicalType() != null ? schema.getLogicalType().getName() : null; - if (logicalType != null) - { - switch (logicalType) - { + if (logicalType != null) { + switch (logicalType) { case "decimal": return TypeDescription.createDecimal( (Integer) (schema.getObjectProp("precision")), @@ -208,8 +180,7 @@ private static TypeDescription parseAvroPrimitive(Schema schema) } } - switch (schema.getType()) - { + switch (schema.getType()) { case LONG: return TypeDescription.createLong(); case INT: diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java index 61b1440..3cd4a62 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionAvroMessageDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import io.apicurio.registry.serde.SerdeConfig; @@ -35,15 +35,13 @@ import java.util.HashMap; import java.util.Map; -public class TransactionAvroMessageDeserializer implements Deserializer -{ +public class TransactionAvroMessageDeserializer implements Deserializer { private static final Logger logger = LoggerFactory.getLogger(TransactionAvroMessageDeserializer.class); private final AvroKafkaDeserializer avroDeserializer = new AvroKafkaDeserializer<>(); private final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); @Override - public void configure(Map configs, boolean isKey) - { + public void configure(Map configs, boolean isKey) { Map enrichedConfig = new HashMap<>(configs); enrichedConfig.put(SerdeConfig.REGISTRY_URL, config.getRegistryUrl()); enrichedConfig.put(SerdeConfig.CHECK_PERIOD_MS, SerdeConfig.CHECK_PERIOD_MS_DEFAULT); @@ -51,27 +49,22 @@ public void configure(Map configs, boolean isKey) } @Override - public SinkProto.TransactionMetadata deserialize(String topic, byte[] bytes) - { - if (bytes == null || bytes.length == 0) - { + public SinkProto.TransactionMetadata deserialize(String topic, byte[] bytes) { + if (bytes == null || bytes.length == 0) { return null; } - try - { + try { MetricsFacade.getInstance().addRawData(bytes.length); GenericRecord avroRecord = avroDeserializer.deserialize(topic, bytes); return TransactionStructMessageDeserializer.convertToTransactionMetadata(avroRecord); - } catch (Exception e) - { + } catch (Exception e) { logger.error("Avro deserialization failed for topic {}: {}", topic, e.getMessage()); throw new SerializationException("Failed to deserialize Avro message", e); } } @Override - public void close() - { + public void close() { Deserializer.super.close(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java index 924b43c..adc33eb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionJsonMessageDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; import com.fasterxml.jackson.databind.ObjectMapper; @@ -31,34 +31,28 @@ import java.io.IOException; import java.util.Map; -public class TransactionJsonMessageDeserializer implements Deserializer -{ +public class TransactionJsonMessageDeserializer implements Deserializer { private static final Logger LOGGER = LoggerFactory.getLogger(TransactionJsonMessageDeserializer.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final JsonFormat.Parser PROTO_PARSER = JsonFormat.parser().ignoringUnknownFields(); @Override - public SinkProto.TransactionMetadata deserialize(String topic, byte[] data) - { - if (data == null || data.length == 0) - { + public SinkProto.TransactionMetadata deserialize(String topic, byte[] data) { + if (data == null || data.length == 0) { return null; } MetricsFacade.getInstance().addRawData(data.length); - try - { + try { Map rawMessage = OBJECT_MAPPER.readValue(data, Map.class); return parseTransactionMetadata(rawMessage); - } catch (IOException e) - { + } catch (IOException e) { LOGGER.error("Failed to deserialize transaction message", e); throw new RuntimeException("Deserialization error", e); } } - private SinkProto.TransactionMetadata parseTransactionMetadata(Map rawMessage) throws IOException - { + private SinkProto.TransactionMetadata parseTransactionMetadata(Map rawMessage) throws IOException { SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); String json = OBJECT_MAPPER.writeValueAsString(rawMessage.get("payload")); PROTO_PARSER.merge(json, builder); diff --git a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java index 9af8c37..be9b61d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/event/deserializer/TransactionStructMessageDeserializer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.event.deserializer; @@ -33,13 +33,11 @@ * @author: AntiO2 * @date: 2025/9/26 12:42 */ -public class TransactionStructMessageDeserializer -{ +public class TransactionStructMessageDeserializer { private static final Logger LOGGER = LoggerFactory.getLogger(TransactionStructMessageDeserializer.class); @SuppressWarnings("unchecked") - public static SinkProto.TransactionMetadata convertToTransactionMetadata(T record) - { + public static SinkProto.TransactionMetadata convertToTransactionMetadata(T record) { SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); builder.setStatus(DeserializerUtil.getStatusSafely(record, "status")) @@ -49,20 +47,16 @@ public static SinkProto.TransactionMetadata convertToTransactionMetadata(T r .setTimestamp(DeserializerUtil.getLongSafely(record, "ts_ms")); Object collections = DeserializerUtil.getFieldSafely(record, "data_collections"); - if (collections instanceof Iterable) - { - for (Object item : (Iterable) collections) - { - if (item instanceof GenericRecord collectionRecord) - { + if (collections instanceof Iterable) { + for (Object item : (Iterable) collections) { + if (item instanceof GenericRecord collectionRecord) { SinkProto.DataCollection.Builder collectionBuilder = SinkProto.DataCollection.newBuilder(); collectionBuilder.setDataCollection( DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); collectionBuilder.setEventCount( DeserializerUtil.getLongSafely(collectionRecord, "event_count")); builder.addDataCollections(collectionBuilder); - } else if (item instanceof Struct collectionRecord) - { + } else if (item instanceof Struct collectionRecord) { SinkProto.DataCollection.Builder collectionBuilder = SinkProto.DataCollection.newBuilder(); collectionBuilder.setDataCollection( DeserializerUtil.getStringSafely(collectionRecord, "data_collection")); diff --git a/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java b/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java index cb19398..aef64a1 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java +++ b/src/main/java/io/pixelsdb/pixels/sink/exception/SinkException.java @@ -17,23 +17,19 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.exception; -public class SinkException extends Exception -{ - public SinkException(String msg) - { +public class SinkException extends Exception { + public SinkException(String msg) { super(msg); } - public SinkException(String message, Throwable cause) - { + public SinkException(String message, Throwable cause) { super(message, cause); } - public SinkException(Throwable cause) - { + public SinkException(Throwable cause) { super(cause); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java index 30f06ce..2ca9fef 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessClient.java @@ -39,8 +39,7 @@ * FreshnessClient is responsible for monitoring data freshness by periodically * querying the maximum timestamp from a set of dynamically configured tables via Trino JDBC. */ -public class FreshnessClient -{ +public class FreshnessClient { private static final Logger LOGGER = LoggerFactory.getLogger(FreshnessClient.class); private static final int QUERY_INTERVAL_SECONDS = 1; private static volatile FreshnessClient instance; @@ -57,8 +56,8 @@ public class FreshnessClient private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private final int warmUpSeconds; private final PixelsSinkConfig config; - private FreshnessClient() - { + + private FreshnessClient() { // Initializes the set with thread safety wrapper this.monitoredTables = Collections.synchronizedSet(new HashSet<>()); @@ -95,15 +94,11 @@ private FreshnessClient() this.connectionExecutor.allowCoreThreadTimeOut(true); } - public static FreshnessClient getInstance() - { - if (instance == null) - { + public static FreshnessClient getInstance() { + if (instance == null) { // First check: Reduces synchronization overhead once the instance is created - synchronized (FreshnessClient.class) - { - if (instance == null) - { + synchronized (FreshnessClient.class) { + if (instance == null) { // Second check: Only one thread proceeds to create the instance instance = new FreshnessClient(); } @@ -113,13 +108,10 @@ public static FreshnessClient getInstance() } @Deprecated - protected Connection createNewConnection() throws SQLException - { - try - { + protected Connection createNewConnection() throws SQLException { + try { Class.forName("io.trino.jdbc.TrinoDriver"); - } catch (ClassNotFoundException e) - { + } catch (ClassNotFoundException e) { throw new SQLException(e); } @@ -129,13 +121,10 @@ protected Connection createNewConnection() throws SQLException return DriverManager.getConnection(trinoJdbcUrl, trinoUser, null); } - protected Connection createNewConnection(long queryTimestamp) throws SQLException - { - try - { + protected Connection createNewConnection(long queryTimestamp) throws SQLException { + try { Class.forName("io.trino.jdbc.TrinoDriver"); - } catch (ClassNotFoundException e) - { + } catch (ClassNotFoundException e) { throw new SQLException(e); } @@ -148,15 +137,11 @@ protected Connection createNewConnection(long queryTimestamp) throws SQLExceptio return DriverManager.getConnection(trinoJdbcUrl, properties); } - private void closeConnection(Connection conn) - { - if (conn != null) - { - try - { + private void closeConnection(Connection conn) { + if (conn != null) { + try { conn.close(); - } catch (SQLException e) - { + } catch (SQLException e) { LOGGER.warn("Error closing Trino connection.", e); } } @@ -172,10 +157,8 @@ private void closeConnection(Connection conn) * * @param tableName The name of the table to add. */ - public void addMonitoredTable(String tableName) - { - if (tableName == null || tableName.trim().isEmpty()) - { + public void addMonitoredTable(String tableName) { + if (tableName == null || tableName.trim().isEmpty()) { LOGGER.warn("Attempted to add null or empty table name to freshness monitor."); return; } @@ -187,13 +170,10 @@ public void addMonitoredTable(String tableName) * * @param tableName The name of the table to remove. */ - public void removeMonitoredTable(String tableName) - { - if (monitoredTables.remove(tableName)) - { + public void removeMonitoredTable(String tableName) { + if (monitoredTables.remove(tableName)) { LOGGER.info("Table '{}' removed from freshness monitor list.", tableName); - } else - { + } else { LOGGER.debug("Table '{}' was not found in the freshness monitor list.", tableName); } } @@ -205,8 +185,7 @@ public void removeMonitoredTable(String tableName) /** * Starts the scheduled freshness monitoring task. */ - public void start() - { + public void start() { LOGGER.info("Starting Freshness Client, querying every {} seconds.", QUERY_INTERVAL_SECONDS); scheduler.scheduleAtFixedRate(this::submitQueryTask, warmUpSeconds, @@ -218,36 +197,29 @@ public void start() /** * Stops the scheduled task and closes the JDBC connection. */ - public void stop() - { + public void stop() { LOGGER.info("Stopping Freshness Client."); scheduler.shutdownNow(); connectionExecutor.shutdownNow(); } - private void submitQueryTask() - { - if (monitoredTables.isEmpty()) - { + private void submitQueryTask() { + if (monitoredTables.isEmpty()) { LOGGER.debug("No tables configured for freshness monitoring. Skipping submission cycle."); return; } - if (!queryPermits.tryAcquire()) - { + if (!queryPermits.tryAcquire()) { LOGGER.debug("Max concurrent queries ({}) reached. Skipping query submission this cycle.", maxConcurrentQueries); return; } - try - { + try { connectionExecutor.submit(this::queryAndCalculateFreshness); - } catch (RejectedExecutionException e) - { + } catch (RejectedExecutionException e) { queryPermits.release(); LOGGER.error("Query task rejected by executor. Max concurrent queries may be too low or service is stopping.", e); - } catch (Exception e) - { + } catch (Exception e) { queryPermits.release(); LOGGER.error("Unknown error during task submission.", e); } @@ -257,30 +229,25 @@ private void submitQueryTask() * The core scheduled task: queries max(freshness_ts) for all monitored tables * and calculates the freshness metric. */ - void queryAndCalculateFreshness() - { + void queryAndCalculateFreshness() { Connection conn = null; TransContext transContext = null; String tableName; - try - { + try { tableName = getRandomTable(); - if(tableName == null) - { + if (tableName == null) { return; } LOGGER.debug("Randomly selected table for this cycle: {}", tableName); // Timestamp when the query is sent (t_send) long tSendMillis = System.currentTimeMillis(); - if(config.isSinkMonitorFreshnessEmbedSnapshot()) - { + if (config.isSinkMonitorFreshnessEmbedSnapshot()) { transContext = TransService.Instance().beginTrans(true); conn = createNewConnection(transContext.getTimestamp()); - } else - { + } else { conn = createNewConnection(); } @@ -290,49 +257,38 @@ void queryAndCalculateFreshness() String query = String.format("SELECT max(freshness_ts) FROM %s WHERE freshness_ts < TIMESTAMP '%s'", tableName, tSendMillisStr); try (Statement statement = conn.createStatement(); - ResultSet rs = statement.executeQuery(query)) - { + ResultSet rs = statement.executeQuery(query)) { Timestamp maxFreshnessTs = null; - if (rs.next()) - { + if (rs.next()) { // Read the maximum timestamp value maxFreshnessTs = rs.getTimestamp(1); } - if (maxFreshnessTs != null) - { + if (maxFreshnessTs != null) { // Freshness = t_send - data_write_time (maxFreshnessTs) // Result is in milliseconds long freshnessMillis = tSendMillis - maxFreshnessTs.getTime(); metricsFacade.recordTableFreshness(tableName, freshnessMillis); - } else - { + } else { LOGGER.warn("Table {} returned null or zero max(freshness_ts). Skipping freshness calculation.", tableName); } - } catch (SQLException e) - { + } catch (SQLException e) { // Handle database errors (e.g., table not found, query syntax error) LOGGER.error("Failed to execute query for table {}: {}", tableName, e.getMessage()); - } catch (Exception e) - { + } catch (Exception e) { // Catch potential runtime errors (e.g., in MetricsFacade) LOGGER.error("Error calculating or recording freshness for table {}.", tableName, e); } - } catch (Exception e) - { + } catch (Exception e) { LOGGER.error("Error selecting a random table from the monitor list.", e); - } finally - { - if(config.isSinkMonitorFreshnessEmbedSnapshot() && transContext != null) - { - try - { + } finally { + if (config.isSinkMonitorFreshnessEmbedSnapshot() && transContext != null) { + try { TransService.Instance().commitTrans(transContext.getTransId(), true); - } catch (TransException e) - { + } catch (TransException e) { throw new RuntimeException(e); } } @@ -343,19 +299,15 @@ void queryAndCalculateFreshness() } - private String getRandomTable() - { + private String getRandomTable() { List tableList; - if(config.isSinkMonitorFreshnessEmbedStatic()) - { + if (config.isSinkMonitorFreshnessEmbedStatic()) { tableList = getStaticList(); - } else - { + } else { tableList = getDynamicList(); } - if(tableList == null || tableList.isEmpty()) - { + if (tableList == null || tableList.isEmpty()) { return null; } @@ -365,14 +317,12 @@ private String getRandomTable() return tableList.get(randomIndex); } - private List getDynamicList() - { + private List getDynamicList() { // Take a snapshot of the tables to monitor for this cycle. // This prevents ConcurrentModificationException if a table is added/removed mid-iteration. Set tablesSnapshot = new HashSet<>(monitoredTables); - if (tablesSnapshot.isEmpty()) - { + if (tablesSnapshot.isEmpty()) { LOGGER.debug("No tables configured for freshness monitoring. Skipping cycle."); return null; } @@ -382,8 +332,7 @@ private List getDynamicList() List staticList = getStaticList(); // If staticList is empty or null, return all tablesSnapshot - if (staticList == null || staticList.isEmpty()) - { + if (staticList == null || staticList.isEmpty()) { return new ArrayList<>(tablesSnapshot); } @@ -394,8 +343,7 @@ private List getDynamicList() return new ArrayList<>(tablesSnapshot); } - private List getStaticList() - { + private List getStaticList() { return config.getSinkMonitorFreshnessEmbedTableList(); } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java index 004c471..613724d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/FreshnessHistory.java @@ -29,22 +29,11 @@ public class FreshnessHistory { private final ConcurrentLinkedQueue history = new ConcurrentLinkedQueue<>(); - public record Record(long timestamp, double value) - { - - @Override - public String toString() - { - return timestamp + "," + value; - } - } - public void record(double freshnessMill) { history.offer(new Record(System.currentTimeMillis(), freshnessMill)); } - public List pollAll() - { + public List pollAll() { if (history.isEmpty()) { return Collections.emptyList(); } @@ -55,4 +44,12 @@ public List pollAll() } return records; } + + public record Record(long timestamp, double value) { + + @Override + public String toString() { + return timestamp + "," + value; + } + } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java b/src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java index 01123e8..c2aeb49 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java +++ b/src/main/java/io/pixelsdb/pixels/sink/freshness/OneSecondAverage.java @@ -17,14 +17,13 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.freshness; import java.util.ArrayDeque; import java.util.Deque; -public class OneSecondAverage -{ +public class OneSecondAverage { /** * Time window in milliseconds */ @@ -38,16 +37,14 @@ public class OneSecondAverage /** * Constructor with configurable window size (milliseconds) */ - public OneSecondAverage(int windowMillis) - { + public OneSecondAverage(int windowMillis) { this.windowMillis = windowMillis; } /** * Record a new data point */ - public synchronized void record(double v) - { + public synchronized void record(double v) { long now = System.currentTimeMillis(); window.addLast(new TimedValue(now, v)); evictOld(now); @@ -56,10 +53,8 @@ public synchronized void record(double v) /** * Remove all values older than windowMillis */ - private void evictOld(long now) - { - while (!window.isEmpty() && now - window.peekFirst().timestamp > windowMillis) - { + private void evictOld(long now) { + while (!window.isEmpty() && now - window.peekFirst().timestamp > windowMillis) { window.removeFirst(); } } @@ -67,19 +62,16 @@ private void evictOld(long now) /** * Compute average of values in the time window */ - public synchronized double getWindowAverage() - { + public synchronized double getWindowAverage() { long now = System.currentTimeMillis(); evictOld(now); - if (window.isEmpty()) - { + if (window.isEmpty()) { return Double.NaN; } double sum = 0; - for (TimedValue tv : window) - { + for (TimedValue tv : window) { sum += tv.value; } return sum / window.size(); @@ -88,7 +80,6 @@ public synchronized double getWindowAverage() /** * Timestamped data point */ - private record TimedValue(long timestamp, double value) - { + private record TimedValue(long timestamp, double value) { } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java index 25d649d..510dced 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadata.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.metadata; import io.pixelsdb.pixels.common.exception.MetadataException; @@ -34,16 +34,14 @@ import java.util.stream.Collectors; @Getter -public class TableMetadata -{ +public class TableMetadata { private final Table table; private final SinglePointIndex index; private final TypeDescription typeDescription; private final List columns; private final List keyColumnNames; - public TableMetadata(Table table, SinglePointIndex index, List columns) throws MetadataException - { + public TableMetadata(Table table, SinglePointIndex index, List columns) throws MetadataException { this.table = table; this.index = index; this.columns = columns; @@ -51,22 +49,17 @@ public TableMetadata(Table table, SinglePointIndex index, List columns) List columnNames = columns.stream().map(Column::getName).collect(Collectors.toList()); List columnTypes = columns.stream().map(Column::getType).collect(Collectors.toList()); typeDescription = TypeDescription.createSchemaFromStrings(columnNames, columnTypes); - if (index != null) - { + if (index != null) { Map columnMap = new HashMap<>(); - for (Column column : columns) - { + for (Column column : columns) { columnMap.put(column.getId(), column); } - for (Integer keyColumnId : index.getKeyColumns().getKeyColumnIds()) - { + for (Integer keyColumnId : index.getKeyColumns().getKeyColumnIds()) { Column column = columnMap.get(keyColumnId.longValue()); - if (column != null) - { + if (column != null) { keyColumnNames.add(column.getName()); - } else - { + } else { throw new MetadataException("Cant find key column id: " + keyColumnId + " in table " + table.getName() + " schema id is " + table.getSchemaId()); } @@ -74,23 +67,19 @@ public TableMetadata(Table table, SinglePointIndex index, List columns) } } - public boolean hasPrimaryIndex() - { + public boolean hasPrimaryIndex() { return index != null; } - public long getPrimaryIndexKeyId() - { + public long getPrimaryIndexKeyId() { return index.getId(); } - public long getTableId() - { + public long getTableId() { return table.getId(); } - public long getSchemaId() - { + public long getSchemaId() { return table.getSchemaId(); } } \ No newline at end of file diff --git a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java index ad3a03c..c754e8e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/metadata/TableMetadataRegistry.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.metadata; import io.pixelsdb.pixels.common.exception.MetadataException; @@ -36,8 +36,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -public class TableMetadataRegistry -{ +public class TableMetadataRegistry { private static final Logger logger = LoggerFactory.getLogger(TableMetadataRegistry.class); private static final MetadataService metadataService = MetadataService.Instance(); @@ -47,18 +46,13 @@ public class TableMetadataRegistry private final ConcurrentMap tableId2SchemaTableName = new ConcurrentHashMap<>(); private List schemas; - private TableMetadataRegistry() - { + private TableMetadataRegistry() { } - public static TableMetadataRegistry Instance() - { - if (instance == null) - { - synchronized (TableMetadataRegistry.class) - { - if (instance == null) - { + public static TableMetadataRegistry Instance() { + if (instance == null) { + synchronized (TableMetadataRegistry.class) { + if (instance == null) { instance = new TableMetadataRegistry(); } } @@ -66,11 +60,9 @@ public static TableMetadataRegistry Instance() return instance; } - public TableMetadata getMetadata(String schema, String table) throws SinkException - { + public TableMetadata getMetadata(String schema, String table) throws SinkException { SchemaTableName key = new SchemaTableName(schema, table); - if (!registry.containsKey(key)) - { + if (!registry.containsKey(key)) { logger.debug("Registry doesn't contain {}", key); TableMetadata metadata = loadTableMetadata(schema, table); registry.put(key, metadata); @@ -79,10 +71,8 @@ public TableMetadata getMetadata(String schema, String table) throws SinkExcepti } - public SchemaTableName getSchemaTableName(long tableId) throws SinkException - { - if (!tableId2SchemaTableName.containsKey(tableId)) - { + public SchemaTableName getSchemaTableName(long tableId) throws SinkException { + if (!tableId2SchemaTableName.containsKey(tableId)) { logger.info("SchemaTableName doesn't contain {}", tableId); SchemaTableName metadata = loadSchemaTableName(tableId); tableId2SchemaTableName.put(tableId, metadata); @@ -90,62 +80,49 @@ public SchemaTableName getSchemaTableName(long tableId) throws SinkException return tableId2SchemaTableName.get(tableId); } - public TypeDescription getTypeDescription(String schemaName, String tableName) throws SinkException - { + public TypeDescription getTypeDescription(String schemaName, String tableName) throws SinkException { return getMetadata(schemaName, tableName).getTypeDescription(); } - public List getKeyColumnsName(String schemaName, String tableName) throws SinkException - { + public List getKeyColumnsName(String schemaName, String tableName) throws SinkException { return getMetadata(schemaName, tableName).getKeyColumnNames(); } - public long getPrimaryIndexKeyId(String schemaName, String tableName) throws SinkException - { + public long getPrimaryIndexKeyId(String schemaName, String tableName) throws SinkException { return getMetadata(schemaName, tableName).getPrimaryIndexKeyId(); } - public long getTableId(String schemaName, String tableName) throws SinkException - { + public long getTableId(String schemaName, String tableName) throws SinkException { return getMetadata(schemaName, tableName).getTableId(); } - private TableMetadata loadTableMetadata(String schemaName, String tableName) throws SinkException - { - try - { + private TableMetadata loadTableMetadata(String schemaName, String tableName) throws SinkException { + try { logger.info("Metadata Cache miss: {} {}", schemaName, tableName); Table table = metadataService.getTable(schemaName, tableName); SinglePointIndex index = null; - try - { + try { index = metadataService.getPrimaryIndex(table.getId()); - } catch (MetadataException e) - { + } catch (MetadataException e) { logger.warn("Could not get primary index for table {}", tableName, e); } - if (!index.isUnique()) - { + if (!index.isUnique()) { throw new MetadataException("Non Unique Index is not supported, Schema:" + schemaName + " Table: " + tableName); } List tableColumns = metadataService.getColumns(schemaName, tableName, false); return new TableMetadata(table, index, tableColumns); - } catch (MetadataException e) - { + } catch (MetadataException e) { throw new SinkException(e); } } - private SchemaTableName loadSchemaTableName(long tableId) throws SinkException - { + private SchemaTableName loadSchemaTableName(long tableId) throws SinkException { // metadataService - try - { - if (schemas == null) - { + try { + if (schemas == null) { schemas = metadataService.getSchemas(); } Table table = metadataService.getTableById(tableId); @@ -159,8 +136,7 @@ private SchemaTableName loadSchemaTableName(long tableId) throws SinkException return new SchemaTableName(table.getName(), schema.getName()); - } catch (MetadataException e) - { + } catch (MetadataException e) { throw new SinkException(e); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java b/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java index b5f78c9..4b39a9a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/MonitorThreadManager.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; @@ -28,50 +28,40 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -public class MonitorThreadManager -{ +public class MonitorThreadManager { private final List monitors = new CopyOnWriteArrayList<>(); private final ExecutorService executor = Executors.newFixedThreadPool(PixelsSinkConstants.MONITOR_NUM); - public void startMonitor(Runnable monitor) - { + public void startMonitor(Runnable monitor) { monitors.add(monitor); executor.submit(monitor); } - public void shutdown() - { + public void shutdown() { stopMonitors(); shutdownExecutor(); awaitTermination(); } - private void stopMonitors() - { + private void stopMonitors() { monitors.forEach(monitor -> { - if (monitor instanceof StoppableProcessor) - { + if (monitor instanceof StoppableProcessor) { ((StoppableProcessor) monitor).stopProcessor(); } }); } - private void shutdownExecutor() - { + private void shutdownExecutor() { executor.shutdown(); } - private void awaitTermination() - { - try - { - if (!executor.awaitTermination(10, TimeUnit.SECONDS)) - { + private void awaitTermination() { + try { + if (!executor.awaitTermination(10, TimeUnit.SECONDS)) { executor.shutdownNow(); } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { executor.shutdownNow(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java index b35e022..4e6beff 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/StoppableProcessor.java @@ -17,11 +17,10 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.processor; -public interface StoppableProcessor -{ +public interface StoppableProcessor { void stopProcessor(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java index 2ee4e80..1d31a28 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TableProcessor.java @@ -17,15 +17,11 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.processor; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; -import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.freshness.FreshnessClient; -import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.provider.TableEventProvider; import io.pixelsdb.pixels.sink.util.MetricsFacade; import io.pixelsdb.pixels.sink.writer.PixelsSinkWriter; @@ -41,8 +37,7 @@ * @author: AntiO2 * @date: 2025/9/26 11:01 */ -public class TableProcessor implements StoppableProcessor, Runnable -{ +public class TableProcessor implements StoppableProcessor, Runnable { private static final Logger LOGGER = LoggerFactory.getLogger(TableProcessor.class); private final AtomicBoolean running = new AtomicBoolean(true); private final PixelsSinkWriter pixelsSinkWriter; @@ -50,26 +45,22 @@ public class TableProcessor implements StoppableProcessor, Runnable private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); private Thread processorThread; private boolean tableAdded = false; - public TableProcessor(TableEventProvider tableEventProvider) - { + + public TableProcessor(TableEventProvider tableEventProvider) { this.pixelsSinkWriter = PixelsSinkWriterFactory.getWriter(); this.tableEventProvider = tableEventProvider; } @Override - public void run() - { + public void run() { processorThread = new Thread(this::processLoop); processorThread.start(); } - private void processLoop() - { - while (running.get()) - { + private void processLoop() { + while (running.get()) { RowChangeEvent event = tableEventProvider.getRowChangeEvent(); - if (event == null) - { + if (event == null) { continue; } pixelsSinkWriter.writeRow(event); @@ -78,8 +69,7 @@ private void processLoop() } @Override - public void stopProcessor() - { + public void stopProcessor() { LOGGER.info("Stopping transaction monitor"); running.set(false); processorThread.interrupt(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java index 5069236..1507f2d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TopicProcessor.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -35,8 +35,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -public class TopicProcessor implements StoppableProcessor, Runnable -{ +public class TopicProcessor implements StoppableProcessor, Runnable { private static final Logger log = LoggerFactory.getLogger(TopicProcessor.class); private final Properties kafkaProperties; @@ -52,8 +51,7 @@ public class TopicProcessor implements StoppableProcessor, Runnable private AdminClient adminClient; private Timer timer; - public TopicProcessor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) - { + public TopicProcessor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaProperties) { this.pixelsSinkConfig = pixelsSinkConfig; this.kafkaProperties = kafkaProperties; this.baseTopic = pixelsSinkConfig.getTopicPrefix() + "." + pixelsSinkConfig.getCaptureDatabase(); @@ -61,30 +59,25 @@ public TopicProcessor(PixelsSinkConfig pixelsSinkConfig, Properties kafkaPropert this.bootstrapServers = pixelsSinkConfig.getBootstrapServers(); } - private static Set filterTopics(Set topics, String prefix) - { + private static Set filterTopics(Set topics, String prefix) { return topics.stream() .filter(t -> t.startsWith(prefix)) .collect(Collectors.toSet()); } @Override - public void run() - { - try - { + public void run() { + try { initializeResources(); startMonitoringCycle(); - } finally - { + } finally { cleanupResources(); log.info("Topic monitor stopped"); } } @Override - public void stopProcessor() - { + public void stopProcessor() { log.info("Initiating topic monitor shutdown..."); running.set(false); interruptMonitoring(); @@ -92,8 +85,7 @@ public void stopProcessor() awaitTermination(); } - private void shutdownConsumerTasks() - { + private void shutdownConsumerTasks() { log.info("Shutting down {} active consumer tasks", activeTasks.size()); activeTasks.forEach((topic, task) -> { @@ -103,23 +95,18 @@ private void shutdownConsumerTasks() activeTasks.clear(); } - private void awaitTermination() - { - try - { - if (executorService != null && !executorService.awaitTermination(30, TimeUnit.SECONDS)) - { + private void awaitTermination() { + try { + if (executorService != null && !executorService.awaitTermination(30, TimeUnit.SECONDS)) { log.warn("Forcing shutdown of remaining tasks"); executorService.shutdownNow(); } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } - private void initializeResources() - { + private void initializeResources() { Properties props = new Properties(); props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); this.adminClient = AdminClient.create(props); @@ -127,20 +114,15 @@ private void initializeResources() log.info("Started topic monitor for base topic: {}", baseTopic); } - private void startMonitoringCycle() - { + private void startMonitoringCycle() { String topicPrefix = baseTopic + "."; timer.scheduleAtFixedRate(new TopicMonitorTask(), 0, 5000); - while (running.get()) - { - try - { + while (running.get()) { + try { TimeUnit.SECONDS.sleep(1); - } catch (InterruptedException e) - { - if (running.get()) - { + } catch (InterruptedException e) { + if (running.get()) { log.warn("Monitoring thread interrupted unexpectedly", e); } Thread.currentThread().interrupt(); @@ -148,133 +130,105 @@ private void startMonitoringCycle() } } - private void interruptMonitoring() - { - if (timer != null) - { + private void interruptMonitoring() { + if (timer != null) { timer.cancel(); timer.purge(); } - if (adminClient != null) - { + if (adminClient != null) { adminClient.close(Duration.ofSeconds(5)); } shutdownExecutorService(); } - private void shutdownExecutorService() - { + private void shutdownExecutorService() { executorService.shutdown(); - try - { - if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) - { + try { + if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) { executorService.shutdownNow(); } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { executorService.shutdownNow(); Thread.currentThread().interrupt(); } } - private void cleanupResources() - { - try - { - if (adminClient != null) - { + private void cleanupResources() { + try { + if (adminClient != null) { adminClient.close(Duration.ofSeconds(5)); } - } catch (Exception e) - { + } catch (Exception e) { log.warn("Error closing admin client", e); } } - private Set detectNewTopics(Set currentTopics) - { + private Set detectNewTopics(Set currentTopics) { return currentTopics.stream() .filter(t -> !subscribedTopics.contains(t)) .collect(Collectors.toSet()); } - private String extractTableName(String topic) - { + private String extractTableName(String topic) { int lastDotIndex = topic.lastIndexOf('.'); return lastDotIndex != -1 ? topic.substring(lastDotIndex + 1) : topic; } - private void launchConsumerTask(String topic) - { - try - { + private void launchConsumerTask(String topic) { + try { TableEventKafkaProvider task = new TableEventKafkaProvider(kafkaProperties, topic); executorService.submit(task); - } catch (Exception e) - { + } catch (Exception e) { log.error("Failed to start consumer for topic {}: {}", topic, e.getMessage()); } } - private class TopicMonitorTask extends TimerTask - { + private class TopicMonitorTask extends TimerTask { @Override - public void run() - { - if (!running.get()) - { + public void run() { + if (!running.get()) { cancel(); return; } - try - { + try { processTopicChanges(); - } catch (Exception e) - { + } catch (Exception e) { e.printStackTrace(); log.error("Error processing topic changes: {}", e.getMessage()); } } - private void processTopicChanges() - { - try - { + private void processTopicChanges() { + try { ListTopicsResult listTopicsResult = adminClient.listTopics(); Set currentTopics = listTopicsResult.names().get(5, TimeUnit.SECONDS); Set filteredTopics = filterTopics(currentTopics, baseTopic + "."); Set newTopics = detectNewTopics(filteredTopics); handleNewTopics(newTopics); - } catch (TimeoutException | ExecutionException | InterruptedException ignored) - { + } catch (TimeoutException | ExecutionException | InterruptedException ignored) { } } - private void handleNewTopics(Set newTopics) - { + private void handleNewTopics(Set newTopics) { newTopics.stream() .filter(this::shouldProcessTable) .forEach(topic -> { - try - { + try { TableEventKafkaProvider task = new TableEventKafkaProvider(kafkaProperties, topic); executorService.submit(task); activeTasks.put(topic, task); subscribedTopics.add(topic); - } catch (IOException e) - { + } catch (IOException e) { log.error("Failed to create consumer for {}: {}", topic, e.getMessage()); } }); } - private boolean shouldProcessTable(String topic) - { + private boolean shouldProcessTable(String topic) { String tableName = extractTableName(topic); return includeTables.length == 0 || Arrays.stream(includeTables).anyMatch(t -> t.equals(tableName)); diff --git a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java index 1841168..a76530c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java +++ b/src/main/java/io/pixelsdb/pixels/sink/processor/TransactionProcessor.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.processor; import io.pixelsdb.pixels.sink.SinkProto; @@ -29,27 +29,22 @@ import java.util.concurrent.atomic.AtomicBoolean; -public class TransactionProcessor implements Runnable, StoppableProcessor -{ +public class TransactionProcessor implements Runnable, StoppableProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(TransactionProcessor.class); private final PixelsSinkWriter sinkWriter; private final AtomicBoolean running = new AtomicBoolean(true); private final TransactionEventProvider transactionEventProvider; - public TransactionProcessor(TransactionEventProvider transactionEventProvider) - { + public TransactionProcessor(TransactionEventProvider transactionEventProvider) { this.transactionEventProvider = transactionEventProvider; this.sinkWriter = PixelsSinkWriterFactory.getWriter(); } @Override - public void run() - { - while (running.get()) - { + public void run() { + while (running.get()) { SinkProto.TransactionMetadata transaction = transactionEventProvider.getTransaction(); - if(transaction == null) - { + if (transaction == null) { LOGGER.warn("Received null transaction"); running.set(false); break; @@ -60,8 +55,7 @@ public void run() } @Override - public void stopProcessor() - { + public void stopProcessor() { LOGGER.info("Stopping transaction monitor"); running.set(false); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java index b520c37..1dd4a6a 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/EventProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.config.PixelsSinkConstants; @@ -30,8 +30,7 @@ import java.util.List; import java.util.concurrent.*; -public abstract class EventProvider implements Runnable, Closeable -{ +public abstract class EventProvider implements Runnable, Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(EventProvider.class); private static final int BATCH_SIZE = 64; @@ -46,26 +45,21 @@ public abstract class EventProvider implements @Override - public void run() - { + public void run() { providerThread = new Thread(this::processLoop); providerThread.start(); } @Override - public void close() - { + public void close() { this.providerThread.interrupt(); decodeExecutor.shutdown(); } - protected void processLoop() - { + protected void processLoop() { List sourceBatch = new ArrayList<>(BATCH_SIZE); - while (true) - { - try - { + while (true) { + try { sourceBatch.clear(); // take first element (blocking) SOURCE_RECORD_T first = getRawEvent(); @@ -73,18 +67,15 @@ protected void processLoop() long startTime = System.nanoTime(); // keep polling until sourceBatch full or timeout - while (sourceBatch.size() < BATCH_SIZE) - { + while (sourceBatch.size() < BATCH_SIZE) { long elapsedMs = (System.nanoTime() - startTime) / 1_000_000; long remainingMs = MAX_WAIT_MS - elapsedMs; - if (remainingMs <= 0) - { + if (remainingMs <= 0) { break; } SOURCE_RECORD_T next = pollRawEvent(remainingMs); - if (next == null) - { + if (next == null) { break; } sourceBatch.add(next); @@ -92,30 +83,24 @@ protected void processLoop() // parallel decode List> futures = new ArrayList<>(sourceBatch.size()); - for (SOURCE_RECORD_T data : sourceBatch) - { + for (SOURCE_RECORD_T data : sourceBatch) { futures.add(decodeExecutor.submit(() -> convertToTargetRecord(data))); } // ordered put into queue - for (Future future : futures) - { - try - { + for (Future future : futures) { + try { TARGET_RECORD_T event = future.get(); - if (event != null) - { + if (event != null) { recordSerdEvent(); putTargetEvent(event); } - } catch (ExecutionException e) - { + } catch (ExecutionException e) { LOGGER.warn("Decode failed: {}", String.valueOf(e.getCause())); } } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; } @@ -124,59 +109,44 @@ protected void processLoop() abstract TARGET_RECORD_T convertToTargetRecord(SOURCE_RECORD_T record); - protected TARGET_RECORD_T getTargetEvent() - { - try - { + protected TARGET_RECORD_T getTargetEvent() { + try { return eventQueue.take(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } return null; } - protected void putTargetEvent(TARGET_RECORD_T event) - { - try - { + protected void putTargetEvent(TARGET_RECORD_T event) { + try { eventQueue.put(event); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } - protected void putRawEvent(SOURCE_RECORD_T record) - { - try - { + protected void putRawEvent(SOURCE_RECORD_T record) { + try { rawEventQueue.put(record); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } - protected SOURCE_RECORD_T getRawEvent() - { - try - { + protected SOURCE_RECORD_T getRawEvent() { + try { return rawEventQueue.take(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); return null; } } - protected SOURCE_RECORD_T pollRawEvent(long remainingMs) - { - try - { + protected SOURCE_RECORD_T pollRawEvent(long remainingMs) { + try { return rawEventQueue.poll(remainingMs, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { return null; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java index 5841614..a854ebe 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/ProtoType.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; @@ -27,32 +27,26 @@ * @author: AntiO2 * @date: 2025/10/5 12:56 */ -public enum ProtoType -{ +public enum ProtoType { ROW(0), TRANS(1); private final int value; - ProtoType(int value) - { + ProtoType(int value) { this.value = value; } - public static ProtoType fromInt(int value) - { - for (ProtoType type : ProtoType.values()) - { - if (type.value == value) - { + public static ProtoType fromInt(int value) { + for (ProtoType type : ProtoType.values()) { + if (type.value == value) { return type; } } throw new IllegalArgumentException("Unknown ProtoType value: " + value); } - public int toInt() - { + public int toInt() { return value; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java index 494d518..5f5d110 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventEngineProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; @@ -35,19 +35,15 @@ * @author: AntiO2 * @date: 2025/9/26 10:45 */ -public class TableEventEngineProvider extends TableEventProvider -{ +public class TableEventEngineProvider extends TableEventProvider { private final Logger LOGGER = LoggerFactory.getLogger(TableEventEngineProvider.class.getName()); @Override - RowChangeEvent convertToTargetRecord(T record) - { + RowChangeEvent convertToTargetRecord(T record) { SourceRecord sourceRecord = (SourceRecord) record; - try - { + try { return RowChangeEventStructDeserializer.convertToRowChangeEvent(sourceRecord); - } catch (SinkException e) - { + } catch (SinkException e) { LOGGER.warn("Failed to convert RowChangeEvent to RowChangeEventStruct {}", e.getMessage()); return null; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java index cef2299..4b32bb0 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventKafkaProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -38,8 +38,7 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; -public class TableEventKafkaProvider extends TableEventProvider -{ +public class TableEventKafkaProvider extends TableEventProvider { private static final Logger log = LoggerFactory.getLogger(TableEventKafkaProvider.class); private final Properties kafkaProperties; private final String topic; @@ -47,8 +46,7 @@ public class TableEventKafkaProvider extends TableEventProvider private final String tableName; private KafkaConsumer consumer; - public TableEventKafkaProvider(Properties kafkaProperties, String topic) throws IOException - { + public TableEventKafkaProvider(Properties kafkaProperties, String topic) throws IOException { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.kafkaProperties = kafkaProperties; this.topic = topic; @@ -59,47 +57,36 @@ public TableEventKafkaProvider(Properties kafkaProperties, String topic) throws } @Override - protected void processLoop() - { - try - { + protected void processLoop() { + try { consumer = new KafkaConsumer<>(kafkaProperties); consumer.subscribe(Collections.singleton(topic)); - while (running.get()) - { - try - { + while (running.get()) { + try { ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (!records.isEmpty()) - { + if (!records.isEmpty()) { log.info("{} Consumer poll returned {} records", tableName, records.count()); records.forEach(record -> { - if (record.value() == null) - { + if (record.value() == null) { return; } metricsFacade.recordSerdRowChange(); putRowChangeEvent(record.value()); }); } - } catch (InterruptException ignored) - { + } catch (InterruptException ignored) { Thread.currentThread().interrupt(); break; } } - } catch (WakeupException e) - { + } catch (WakeupException e) { log.info("Consumer wakeup triggered for {}", tableName); - } catch (Exception e) - { + } catch (Exception e) { log.info("Exception: {}", e.getMessage()); - } finally - { - if (consumer != null) - { + } finally { + if (consumer != null) { consumer.close(Duration.ofSeconds(5)); log.info("Kafka consumer closed for {}", tableName); } @@ -107,8 +94,7 @@ protected void processLoop() } @Override - RowChangeEvent convertToTargetRecord(Void record) - { + RowChangeEvent convertToTargetRecord(Void record) { throw new UnsupportedOperationException(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java index 73f6d66..2b53699 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; @@ -29,25 +29,20 @@ * @author: AntiO2 * @date: 2025/9/26 07:47 */ -public abstract class TableEventProvider extends EventProvider -{ - protected void putRowChangeEvent(RowChangeEvent rowChangeEvent) - { +public abstract class TableEventProvider extends EventProvider { + protected void putRowChangeEvent(RowChangeEvent rowChangeEvent) { putTargetEvent(rowChangeEvent); } - public RowChangeEvent getRowChangeEvent() - { + public RowChangeEvent getRowChangeEvent() { return getTargetEvent(); } - protected void putRawRowChangeEvent(SOURCE_RECORD_T record) - { + protected void putRawRowChangeEvent(SOURCE_RECORD_T record) { putRawEvent(record); } - final protected void recordSerdEvent() - { + final protected void recordSerdEvent() { metricsFacade.recordSerdRowChange(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java index 949131e..1676e20 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageLoopProvider.java @@ -17,59 +17,49 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import com.google.protobuf.InvalidProtocolBufferException; +import io.pixelsdb.pixels.core.utils.Pair; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.event.deserializer.RowChangeEventStructDeserializer; import io.pixelsdb.pixels.sink.exception.SinkException; +import io.pixelsdb.pixels.sink.util.DataTransform; import java.nio.ByteBuffer; import java.util.logging.Logger; -import io.pixelsdb.pixels.core.utils.Pair; -import io.pixelsdb.pixels.sink.freshness.FreshnessClient; -import io.pixelsdb.pixels.sink.util.DataTransform; - -import javax.xml.crypto.Data; -public class TableEventStorageLoopProvider extends TableEventProvider -{ +public class TableEventStorageLoopProvider extends TableEventProvider { private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); private final boolean freshness_embed; private final boolean freshness_timestamp; - protected TableEventStorageLoopProvider() - { + protected TableEventStorageLoopProvider() { super(); PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); String sinkMonitorFreshnessLevel = config.getSinkMonitorFreshnessLevel(); - if(sinkMonitorFreshnessLevel.equals("embed")) - { + if (sinkMonitorFreshnessLevel.equals("embed")) { freshness_embed = true; - } else - { + } else { freshness_embed = false; } freshness_timestamp = config.isSinkMonitorFreshnessTimestamp(); } @Override - RowChangeEvent convertToTargetRecord(T record) - { + RowChangeEvent convertToTargetRecord(T record) { Pair pairRecord = (Pair) record; ByteBuffer sourceRecord = pairRecord.getLeft(); Integer loopId = pairRecord.getRight(); - try - { + try { SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); SinkProto.RowRecord.Builder rowRecordBuilder = rowRecord.toBuilder(); - if(freshness_timestamp) - { + if (freshness_timestamp) { DataTransform.updateRecordTimestamp(rowRecordBuilder, System.currentTimeMillis() * 1000); } @@ -78,8 +68,7 @@ RowChangeEvent convertToTargetRecord(T record) transactionBuilder.setId(id + "_" + loopId); rowRecordBuilder.setTransaction(transactionBuilder); return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecordBuilder.build()); - } catch (InvalidProtocolBufferException | SinkException e) - { + } catch (InvalidProtocolBufferException | SinkException e) { LOGGER.warning(e.getMessage()); return null; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java index 6703625..17a37b3 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableEventStorageProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; @@ -36,25 +36,20 @@ * @author: AntiO2 * @date: 2025/9/26 10:45 */ -public class TableEventStorageProvider extends TableEventProvider -{ +public class TableEventStorageProvider extends TableEventProvider { private final Logger LOGGER = Logger.getLogger(TableEventStorageProvider.class.getName()); - protected TableEventStorageProvider() - { + protected TableEventStorageProvider() { super(); } @Override - RowChangeEvent convertToTargetRecord(T record) - { + RowChangeEvent convertToTargetRecord(T record) { ByteBuffer sourceRecord = (ByteBuffer) record; - try - { + try { SinkProto.RowRecord rowRecord = SinkProto.RowRecord.parseFrom(sourceRecord); return RowChangeEventStructDeserializer.convertToRowChangeEvent(rowRecord); - } catch (InvalidProtocolBufferException | SinkException e) - { + } catch (InvalidProtocolBufferException | SinkException e) { LOGGER.warning(e.getMessage()); return null; } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java index 6e8d9c3..8983c21 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TableProviderAndProcessorPipelineManager.java @@ -17,18 +17,12 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.common.metadata.SchemaTableName; import io.pixelsdb.pixels.core.utils.Pair; -import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; -import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.exception.SinkException; -import io.pixelsdb.pixels.sink.freshness.FreshnessClient; -import io.pixelsdb.pixels.sink.metadata.TableMetadata; -import io.pixelsdb.pixels.sink.metadata.TableMetadataRegistry; import io.pixelsdb.pixels.sink.processor.TableProcessor; import org.apache.kafka.connect.source.SourceRecord; @@ -43,21 +37,18 @@ * @author: AntiO2 * @date: 2025/9/26 10:44 */ -public class TableProviderAndProcessorPipelineManager -{ +public class TableProviderAndProcessorPipelineManager { protected final Map activeTableProcessors = new ConcurrentHashMap<>(); protected final Map tableIds = new ConcurrentHashMap<>(); private final Map> tableProviders = new ConcurrentHashMap<>(); private final AtomicInteger nextTableId = new AtomicInteger(); - public void routeRecord(SchemaTableName schemaTableName, SOURCE_RECORD_T record) - { + public void routeRecord(SchemaTableName schemaTableName, SOURCE_RECORD_T record) { routeRecord(getTableId(schemaTableName), record); } - public void routeRecord(Integer tableId, SOURCE_RECORD_T record) - { + public void routeRecord(Integer tableId, SOURCE_RECORD_T record) { TableEventProvider pipeline = tableProviders.computeIfAbsent(tableId, k -> { TableEventProvider newPipeline = createProvider(record); @@ -71,32 +62,25 @@ public void routeRecord(Integer tableId, SOURCE_RECORD_T record) pipeline.putRawEvent(record); } - private TableEventProvider createProvider(SOURCE_RECORD_T record) - { + private TableEventProvider createProvider(SOURCE_RECORD_T record) { Class recordType = record.getClass(); - if (recordType == Pair.class) - { + if (recordType == Pair.class) { return new TableEventStorageLoopProvider<>(); } - if (recordType == SourceRecord.class) - { + if (recordType == SourceRecord.class) { return new TableEventEngineProvider<>(); - } else if (ByteBuffer.class.isAssignableFrom(recordType)) - { + } else if (ByteBuffer.class.isAssignableFrom(recordType)) { return new TableEventStorageProvider<>(); - } else - { + } else { throw new IllegalArgumentException("Unsupported record type: " + recordType.getName()); } } - private Integer getTableId(SchemaTableName schemaTableName) - { + private Integer getTableId(SchemaTableName schemaTableName) { return tableIds.computeIfAbsent(schemaTableName, k -> allocateTableId()); } - private Integer allocateTableId() - { + private Integer allocateTableId() { return nextTableId.getAndIncrement(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java index 57feefc..a0f2ecd 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventEngineProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; @@ -32,19 +32,16 @@ * @author: AntiO2 * @date: 2025/9/25 13:20 */ -public class TransactionEventEngineProvider extends TransactionEventProvider -{ +public class TransactionEventEngineProvider extends TransactionEventProvider { public static final TransactionEventEngineProvider INSTANCE = new TransactionEventEngineProvider<>(); - public static TransactionEventEngineProvider getInstance() - { + public static TransactionEventEngineProvider getInstance() { return INSTANCE; } @Override - SinkProto.TransactionMetadata convertToTargetRecord(T record) - { + SinkProto.TransactionMetadata convertToTargetRecord(T record) { SourceRecord sourceRecord = (SourceRecord) record; Struct value = (Struct) sourceRecord.value(); return TransactionStructMessageDeserializer.convertToTransactionMetadata(value); diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java index bb5f49f..24cf341 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventKafkaProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; @@ -40,14 +40,12 @@ * @author: AntiO2 * @date: 2025/9/25 13:40 */ -public class TransactionEventKafkaProvider extends TransactionEventProvider -{ +public class TransactionEventKafkaProvider extends TransactionEventProvider { private final AtomicBoolean running = new AtomicBoolean(true); private final String transactionTopic; private final KafkaConsumer consumer; - private TransactionEventKafkaProvider() - { + private TransactionEventKafkaProvider() { Properties kafkaProperties = new Properties(); PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); this.transactionTopic = pixelsSinkConfig.getTopicPrefix() + "." + pixelsSinkConfig.getTransactionTopicSuffix(); @@ -56,33 +54,25 @@ private TransactionEventKafkaProvider() @Override - public void processLoop() - { + public void processLoop() { consumer.subscribe(Collections.singletonList(transactionTopic)); - while (running.get()) - { - try - { + while (running.get()) { + try { ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); - for (ConsumerRecord record : records) - { - if (record.value() == null) - { + for (ConsumerRecord record : records) { + if (record.value() == null) { continue; } putTargetEvent(record.value()); } - } catch (WakeupException e) - { - if (running.get()) - { + } catch (WakeupException e) { + if (running.get()) { // LOGGER.warn("Consumer wakeup unexpectedly", e); } - } catch (Exception e) - { + } catch (Exception e) { e.printStackTrace(); throw new RuntimeException(e); } @@ -90,8 +80,7 @@ public void processLoop() } @Override - SinkProto.TransactionMetadata convertToTargetRecord(T record) - { + SinkProto.TransactionMetadata convertToTargetRecord(T record) { throw new UnsupportedOperationException(); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java index 413eb64..7297254 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventProvider.java @@ -17,25 +17,21 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import io.pixelsdb.pixels.sink.SinkProto; -public abstract class TransactionEventProvider extends EventProvider -{ - public void putTransRawEvent(SOURCE_RECORD_T record) - { +public abstract class TransactionEventProvider extends EventProvider { + public void putTransRawEvent(SOURCE_RECORD_T record) { putRawEvent(record); } - public SinkProto.TransactionMetadata getTransaction() - { + public SinkProto.TransactionMetadata getTransaction() { return getTargetEvent(); } - final protected void recordSerdEvent() - { + final protected void recordSerdEvent() { metricsFacade.recordSerdTxChange(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java index d225ea0..9549b3c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageLoopProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import com.google.protobuf.InvalidProtocolBufferException; @@ -26,21 +26,17 @@ import java.nio.ByteBuffer; -public class TransactionEventStorageLoopProvider extends TransactionEventProvider -{ +public class TransactionEventStorageLoopProvider extends TransactionEventProvider { @Override - SinkProto.TransactionMetadata convertToTargetRecord(T record) - { + SinkProto.TransactionMetadata convertToTargetRecord(T record) { Pair buffer = (Pair) record; - try - { + try { SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(buffer.getLeft()); Integer loopId = buffer.getRight(); SinkProto.TransactionMetadata.Builder builder = tx.toBuilder(); builder.setId(builder.getId() + "_" + loopId); return builder.build(); - } catch (InvalidProtocolBufferException e) - { + } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java index 3527268..6a1b9e2 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java +++ b/src/main/java/io/pixelsdb/pixels/sink/provider/TransactionEventStorageProvider.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.provider; import com.google.protobuf.InvalidProtocolBufferException; @@ -25,18 +25,14 @@ import java.nio.ByteBuffer; -public class TransactionEventStorageProvider extends TransactionEventProvider -{ +public class TransactionEventStorageProvider extends TransactionEventProvider { @Override - SinkProto.TransactionMetadata convertToTargetRecord(T record) - { + SinkProto.TransactionMetadata convertToTargetRecord(T record) { ByteBuffer buffer = (ByteBuffer) record; - try - { + try { SinkProto.TransactionMetadata tx = SinkProto.TransactionMetadata.parseFrom(buffer); return tx; - } catch (InvalidProtocolBufferException e) - { + } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java index 274fd93..a96dc56 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source; @@ -30,8 +30,7 @@ * @author: AntiO2 * @date: 2025/9/26 13:45 */ -public interface SinkSource extends StoppableProcessor -{ +public interface SinkSource extends StoppableProcessor { void start(); boolean isRunning(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java index 207edb8..02b293b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/SinkSourceFactory.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source; @@ -27,13 +27,10 @@ import io.pixelsdb.pixels.sink.source.kafka.SinkKafkaSource; import io.pixelsdb.pixels.sink.source.storage.FasterSinkStorageSource; -public class SinkSourceFactory -{ - public static SinkSource createSinkSource() - { +public class SinkSourceFactory { + public static SinkSource createSinkSource() { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - return switch (config.getDataSource()) - { + return switch (config.getDataSource()) { case "kafka" -> new SinkKafkaSource(); case "engine" -> new SinkEngineSource(); case "storage" -> new FasterSinkStorageSource(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java b/src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java index 933473a..715cab6 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/engine/PixelsDebeziumConsumer.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.engine; @@ -42,8 +42,7 @@ * @author: AntiO2 * @date: 2025/9/25 12:51 */ -public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer>, StoppableProcessor -{ +public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer>, StoppableProcessor { private final String checkTransactionTopic; private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; private final TableProviderAndProcessorPipelineManager tableProvidersManagerImpl = new TableProviderAndProcessorPipelineManager<>(); @@ -53,8 +52,7 @@ public class PixelsDebeziumConsumer implements DebeziumEngine.ChangeConsumer> event, - DebeziumEngine.RecordCommitter> committer) throws InterruptedException - { - for (RecordChangeEvent record : event) - { - try - { + DebeziumEngine.RecordCommitter> committer) throws InterruptedException { + for (RecordChangeEvent record : event) { + try { SourceRecord sourceRecord = record.record(); - if (sourceRecord == null) - { + if (sourceRecord == null) { continue; } metricsFacade.recordDebeziumEvent(); - if (isTransactionEvent(sourceRecord)) - { + if (isTransactionEvent(sourceRecord)) { handleTransactionSourceRecord(sourceRecord); - } else - { + } else { handleRowChangeSourceRecord(sourceRecord); } - } finally - { + } finally { committer.markProcessed(record); } @@ -94,13 +85,11 @@ public void handleBatch(List> event, committer.markBatchFinished(); } - private void handleTransactionSourceRecord(SourceRecord sourceRecord) throws InterruptedException - { + private void handleTransactionSourceRecord(SourceRecord sourceRecord) throws InterruptedException { transactionEventProvider.putTransRawEvent(sourceRecord); } - private void handleRowChangeSourceRecord(SourceRecord sourceRecord) - { + private void handleRowChangeSourceRecord(SourceRecord sourceRecord) { Struct value = (Struct) sourceRecord.value(); Struct source = (Struct) value.get("source"); String schemaName = source.get("db").toString(); @@ -109,14 +98,12 @@ private void handleRowChangeSourceRecord(SourceRecord sourceRecord) tableProvidersManagerImpl.routeRecord(schemaTableName, sourceRecord); } - private boolean isTransactionEvent(SourceRecord sourceRecord) - { + private boolean isTransactionEvent(SourceRecord sourceRecord) { return checkTransactionTopic.equals(sourceRecord.topic()); } @Override - public void stopProcessor() - { + public void stopProcessor() { transactionProviderThread.interrupt(); processor.stopProcessor(); transactionProcessorThread.interrupt(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java index a692ea8..c573e40 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/engine/SinkEngineSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.engine; import io.debezium.embedded.Connect; @@ -32,20 +32,17 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -public class SinkEngineSource implements SinkSource -{ +public class SinkEngineSource implements SinkSource { private final PixelsDebeziumConsumer consumer; private DebeziumEngine> engine; private ExecutorService executor; private volatile boolean running = true; - public SinkEngineSource() - { + public SinkEngineSource() { this.consumer = new PixelsDebeziumConsumer(); } - public void start() - { + public void start() { Properties debeziumProps = PixelsSinkConfigFactory.getInstance() .getConfig().extractPropertiesByPrefix("debezium.", true); @@ -59,31 +56,24 @@ public void start() } @Override - public void stopProcessor() - { - try - { - if (engine != null) - { + public void stopProcessor() { + try { + if (engine != null) { engine.close(); } - if (executor != null) - { + if (executor != null) { executor.shutdown(); } consumer.stopProcessor(); - } catch (Exception e) - { + } catch (Exception e) { throw new RuntimeException("Failed to stop PixelsSinkEngine", e); - } finally - { + } finally { running = false; } } @Override - public boolean isRunning() - { + public boolean isRunning() { return running; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java index c82fe1e..0dba2fb 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/kafka/SinkKafkaSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.kafka; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; @@ -31,14 +31,12 @@ import java.util.Properties; -public class SinkKafkaSource implements SinkSource -{ +public class SinkKafkaSource implements SinkSource { private MonitorThreadManager manager; private volatile boolean running = true; @Override - public void start() - { + public void start() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); KafkaPropFactorySelector kafkaPropFactorySelector = new KafkaPropFactorySelector(); @@ -59,15 +57,13 @@ public void start() @Override - public void stopProcessor() - { + public void stopProcessor() { manager.shutdown(); running = false; } @Override - public boolean isRunning() - { + public boolean isRunning() { return running; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java index fc8b857..20b3009 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractMemorySinkStorageSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.storage; import io.pixelsdb.pixels.common.physical.PhysicalReader; @@ -38,8 +38,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; -public abstract class AbstractMemorySinkStorageSource extends AbstractSinkStorageSource -{ +public abstract class AbstractMemorySinkStorageSource extends AbstractSinkStorageSource { private static final Logger LOGGER = LoggerFactory.getLogger(AbstractMemorySinkStorageSource.class); // All preloaded records, order preserved @@ -47,35 +46,29 @@ public abstract class AbstractMemorySinkStorageSource extends AbstractSinkStorag private final List> preloadedRecords = new ArrayList<>(); @Override - public void start() - { + public void start() { this.running.set(true); this.transactionProcessorThread.start(); this.transactionProviderThread.start(); - try - { + try { /* ===================================================== * 1. Initialization phase: preload all ByteBuffers * ===================================================== */ - for (String file : files) - { + for (String file : files) { Storage.Scheme scheme = Storage.Scheme.fromPath(file); LOGGER.info("Preloading file {}", file); PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); readers.add(reader); - while (true) - { + while (true) { int key; int valueLen; - try - { + try { key = reader.readInt(ByteOrder.BIG_ENDIAN); valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException eof) - { + } catch (IOException eof) { // Reached end of file break; } @@ -93,10 +86,8 @@ public void start() * Queue initialization, consumer startup, and feeding * are done together in this phase * ===================================================== */ - do - { - for (Pair record : preloadedRecords) - { + do { + for (Pair record : preloadedRecords) { int key = record.getLeft(); ByteBuffer buffer = record.getRight(); @@ -118,8 +109,7 @@ public void start() }); ProtoType protoType = getProtoType(key); - if (protoType == ProtoType.ROW) - { + if (protoType == ProtoType.ROW) { sourceRateLimiter.acquire(1); } @@ -131,16 +121,11 @@ public void start() } ++loopId; } while (storageLoopEnabled && isRunning()); - } catch (IOException | IndexOutOfBoundsException e) - { + } catch (IOException | IndexOutOfBoundsException e) { throw new RuntimeException(e); - } - catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); - } - finally - { + } finally { clean(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java index 2055cc3..08d7a0d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractReaderSinkStorageSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.storage; import io.pixelsdb.pixels.common.physical.PhysicalReader; @@ -32,54 +32,41 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; -public abstract class AbstractReaderSinkStorageSource extends AbstractSinkStorageSource -{ +public abstract class AbstractReaderSinkStorageSource extends AbstractSinkStorageSource { private static final Logger LOGGER = LoggerFactory.getLogger(AbstractReaderSinkStorageSource.class); @Override - public void start() - { + public void start() { this.running.set(true); this.transactionProcessorThread.start(); this.transactionProviderThread.start(); - for (String file : files) - { + for (String file : files) { Storage.Scheme scheme = Storage.Scheme.fromPath(file); LOGGER.info("Start read from file {}", file); PhysicalReader reader; - try - { + try { reader = PhysicalReaderUtil.newPhysicalReader(scheme, file); - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException(e); } readers.add(reader); } - do - { - for (PhysicalReader reader : readers) - { + do { + for (PhysicalReader reader : readers) { LOGGER.info("Start Read {}", reader.getPath()); long offset = 0; - while (true) - { - try - { + while (true) { + try { int key, valueLen; reader.seek(offset); - try - { + try { key = reader.readInt(ByteOrder.BIG_ENDIAN); valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) - { + } catch (IOException e) { // EOF break; } @@ -99,8 +86,7 @@ public void start() k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE)); // Put future in queue - if(protoType.equals(ProtoType.ROW)) - { + if (protoType.equals(ProtoType.ROW)) { sourceRateLimiter.acquire(1); } queue.put(new Pair<>(valueFuture, loopId)); @@ -112,8 +98,7 @@ public void start() t.start(); return t; }); - } catch (IOException | InterruptedException e) - { + } catch (IOException | InterruptedException e) { break; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java index 66702df..0e4cd9d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/AbstractSinkStorageSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.storage; import io.pixelsdb.pixels.common.physical.PhysicalReader; @@ -41,11 +41,13 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; -public abstract class AbstractSinkStorageSource implements SinkSource -{ +public abstract class AbstractSinkStorageSource implements SinkSource { private static final Logger LOGGER = LoggerFactory.getLogger(AbstractSinkStorageSource.class); protected final AtomicBoolean running = new AtomicBoolean(false); @@ -55,20 +57,20 @@ public abstract class AbstractSinkStorageSource implements SinkSource protected final List files; protected final CompletableFuture POISON_PILL = new CompletableFuture<>(); protected final Map consumerThreads = new ConcurrentHashMap<>(); - private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); protected final Map, Integer>>> queueMap = new ConcurrentHashMap<>(); - private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private final TableProviderAndProcessorPipelineManager> tablePipelineManager = new TableProviderAndProcessorPipelineManager<>(); protected final boolean storageLoopEnabled; protected final FlushRateLimiter sourceRateLimiter; + private final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); + private final TableProviderAndProcessorPipelineManager> tablePipelineManager = new TableProviderAndProcessorPipelineManager<>(); protected TransactionEventStorageLoopProvider> transactionEventProvider; protected TransactionProcessor transactionProcessor; protected Thread transactionProviderThread; protected Thread transactionProcessorThread; protected int loopId = 0; - protected List readers = new ArrayList<>(); - protected AbstractSinkStorageSource() - { + protected List readers = new ArrayList<>(); + + protected AbstractSinkStorageSource() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); this.topic = pixelsSinkConfig.getSinkProtoData(); this.baseDir = pixelsSinkConfig.getSinkProtoDir(); @@ -86,82 +88,62 @@ protected AbstractSinkStorageSource() abstract ProtoType getProtoType(int i); - protected void clean() - { + protected void clean() { queueMap.values().forEach(q -> { - try - { + try { q.put(new Pair<>(POISON_PILL, loopId)); - } - catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); consumerThreads.values().forEach(t -> { - try - { + try { t.join(); - } - catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); - for (PhysicalReader reader : readers) - { - try - { + for (PhysicalReader reader : readers) { + try { reader.close(); - } - catch (IOException e) - { + } catch (IOException e) { LOGGER.warn("Failed to close reader", e); } } } - protected void handleTransactionSourceRecord(ByteBuffer record, Integer loopId) - { + protected void handleTransactionSourceRecord(ByteBuffer record, Integer loopId) { transactionEventProvider.putTransRawEvent(new Pair<>(record, loopId)); } - protected void consumeQueue(int key, BlockingQueue, Integer>> queue, ProtoType protoType) - { - try - { - while (true) - { + protected void consumeQueue(int key, BlockingQueue, Integer>> queue, ProtoType protoType) { + try { + while (true) { Pair, Integer> pair = queue.take(); CompletableFuture value = pair.getLeft(); int loopId = pair.getRight(); - if (value == POISON_PILL) - { + if (value == POISON_PILL) { break; } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - switch (protoType) - { + switch (protoType) { case ROW -> handleRowChangeSourceRecord(key, valueBuffer, loopId); case TRANS -> handleTransactionSourceRecord(valueBuffer, loopId); } } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); - } catch (ExecutionException e) - { + } catch (ExecutionException e) { LOGGER.error("Error in async processing", e); } } - protected ByteBuffer copyToHeap(ByteBuffer directBuffer) - { + protected ByteBuffer copyToHeap(ByteBuffer directBuffer) { ByteBuffer duplicate = directBuffer.duplicate(); ByteBuffer heapBuffer = ByteBuffer.allocate(duplicate.remaining()); heapBuffer.put(duplicate); @@ -169,20 +151,17 @@ protected ByteBuffer copyToHeap(ByteBuffer directBuffer) return heapBuffer; } - protected void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer, int loopId) - { + protected void handleRowChangeSourceRecord(int key, ByteBuffer dataBuffer, int loopId) { tablePipelineManager.routeRecord(key, new Pair<>(dataBuffer, loopId)); } @Override - public boolean isRunning() - { + public boolean isRunning() { return running.get(); } @Override - public void stopProcessor() - { + public void stopProcessor() { running.set(false); transactionProviderThread.interrupt(); transactionProcessorThread.interrupt(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/FasterSinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/FasterSinkStorageSource.java index ae70d6e..6f98c46 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/storage/FasterSinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/FasterSinkStorageSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.storage; @@ -35,28 +35,23 @@ * @author: AntiO2 * @date: 2025/10/5 11:43 */ -public class FasterSinkStorageSource extends AbstractMemorySinkStorageSource implements SinkSource -{ +public class FasterSinkStorageSource extends AbstractMemorySinkStorageSource implements SinkSource { private static final Logger LOGGER = LoggerFactory.getLogger(FasterSinkStorageSource.class); static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); - public FasterSinkStorageSource() - { + public FasterSinkStorageSource() { super(); } - private static String readString(ByteBuffer buffer, int len) - { + private static String readString(ByteBuffer buffer, int len) { byte[] bytes = new byte[len]; buffer.get(bytes); return new String(bytes); } @Override - ProtoType getProtoType(int i) - { - if (i == -1) - { + ProtoType getProtoType(int i) { + if (i == -1) { return ProtoType.TRANS; } return ProtoType.ROW; diff --git a/src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java b/src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java index e8fd71e..567700f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java +++ b/src/main/java/io/pixelsdb/pixels/sink/source/storage/LegacySinkStorageSource.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.source.storage; @@ -49,8 +49,7 @@ * @date: 2025/10/5 11:43 */ @Deprecated -public class LegacySinkStorageSource extends AbstractReaderSinkStorageSource implements SinkSource -{ +public class LegacySinkStorageSource extends AbstractReaderSinkStorageSource implements SinkSource { private static final Logger LOGGER = LoggerFactory.getLogger(LegacySinkStorageSource.class); static SchemaTableName transactionSchemaTableName = new SchemaTableName("freak", "transaction"); private final TransactionEventEngineProvider transactionEventProvider = TransactionEventEngineProvider.INSTANCE; @@ -65,44 +64,35 @@ public class LegacySinkStorageSource extends AbstractReaderSinkStorageSource imp private final CompletableFuture POISON_PILL = new CompletableFuture<>(); - private static String readString(ByteBuffer buffer, int len) - { + private static String readString(ByteBuffer buffer, int len) { byte[] bytes = new byte[len]; buffer.get(bytes); return new String(bytes); } @Override - ProtoType getProtoType(int i) - { + ProtoType getProtoType(int i) { return ProtoType.fromInt(i); } @Override - public void start() - { + public void start() { - for (String file : files) - { + for (String file : files) { Storage.Scheme scheme = Storage.Scheme.fromPath(file); LOGGER.info("Start read from file {}", file); - try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) - { + try (PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(scheme, file)) { long offset = 0; BlockingQueue>> rowQueue = new LinkedBlockingQueue<>(); BlockingQueue> transQueue = new LinkedBlockingQueue<>(); - while (true) - { - try - { + while (true) { + try { int keyLen, valueLen; reader.seek(offset); - try - { + try { keyLen = reader.readInt(ByteOrder.BIG_ENDIAN); valueLen = reader.readInt(ByteOrder.BIG_ENDIAN); - } catch (IOException e) - { + } catch (IOException e) { // EOF break; } @@ -135,13 +125,11 @@ public void start() t.start(); return t; }); - } catch (IOException | InterruptedException e) - { + } catch (IOException | InterruptedException e) { break; } } - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException(e); } @@ -150,11 +138,9 @@ public void start() // signal all queues to stop queueMap.values().forEach(q -> { - try - { + try { q.put(POISON_PILL); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); @@ -162,74 +148,58 @@ public void start() // wait all consumers to finish consumerThreads.values().forEach(t -> { - try - { + try { t.join(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } }); } - private void consumeQueue(SchemaTableName key, BlockingQueue> queue, ProtoType protoType) - { - try - { - while (true) - { + private void consumeQueue(SchemaTableName key, BlockingQueue> queue, ProtoType protoType) { + try { + while (true) { CompletableFuture value = queue.take(); - if (value == POISON_PILL) - { + if (value == POISON_PILL) { break; } ByteBuffer valueBuffer = value.get(); metricsFacade.recordDebeziumEvent(); - switch (protoType) - { + switch (protoType) { case ROW -> handleRowChangeSourceRecord(0, valueBuffer, 0); case TRANS -> handleTransactionSourceRecord(valueBuffer, 0); } } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); - } catch (ExecutionException e) - { + } catch (ExecutionException e) { LOGGER.error("Error in async processing", e); } } - private SchemaTableName computeQueueKey(ByteBuffer keyBuffer, ProtoType protoType) - { - switch (protoType) - { - case ROW -> - { + private SchemaTableName computeQueueKey(ByteBuffer keyBuffer, ProtoType protoType) { + switch (protoType) { + case ROW -> { int schemaLen = keyBuffer.getInt(); int tableLen = keyBuffer.getInt(); String schemaName = readString(keyBuffer, schemaLen); String tableName = readString(keyBuffer, tableLen); return new SchemaTableName(schemaName, tableName); } - case TRANS -> - { + case TRANS -> { return transactionSchemaTableName; } - default -> - { + default -> { throw new IllegalArgumentException("Proto type " + protoType.toString()); } } } - private void handleRowChangeSourceRecord(SchemaTableName schemaTableName, ByteBuffer dataBuffer) - { + private void handleRowChangeSourceRecord(SchemaTableName schemaTableName, ByteBuffer dataBuffer) { tableProvidersManagerImpl.routeRecord(schemaTableName, dataBuffer); } - private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBuffer) - { + private void handleRowChangeSourceRecord(ByteBuffer keyBuffer, ByteBuffer dataBuffer) { { // CODE BLOCK VERSION 2 // long tableId = keyBuffer.getLong(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java index 11b24b8..a48348c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/BlockingBoundedMap.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; import java.util.Set; @@ -33,14 +33,12 @@ * When the map reaches its maximum size, any new insertion or compute * for a new key will block until space becomes available. */ -public class BlockingBoundedMap -{ +public class BlockingBoundedMap { private final int maxSize; private final Semaphore semaphore; private final ConcurrentMap map; - public BlockingBoundedMap(int maxSize) - { + public BlockingBoundedMap(int maxSize) { this.maxSize = maxSize; this.map = new ConcurrentHashMap<>(); this.semaphore = new Semaphore(maxSize); @@ -50,37 +48,31 @@ public BlockingBoundedMap(int maxSize) * Puts a key-value pair into the map. * If the map is full, this call blocks until space becomes available. */ - private void put(K key, V value) throws InterruptedException - { + private void put(K key, V value) throws InterruptedException { semaphore.acquire(); // block if full V prev = map.put(key, value); - if (prev != null) - { + if (prev != null) { // replaced existing value — no new space consumed semaphore.release(); } } - public V get(K key) - { + public V get(K key) { return map.get(key); } /** * Removes a key from the map and releases one permit if a value was present. */ - public V remove(K key) - { + public V remove(K key) { V val = map.remove(key); - if (val != null) - { + if (val != null) { semaphore.release(); } return val; } - public int size() - { + public int size() { return map.size(); } @@ -91,47 +83,37 @@ public int size() * - If the key already exists, it does not block. * - If the remapping function returns null, the key is removed and capacity is released. */ - public V compute(K key, BiFunction remappingFunction) - { - for (; ; ) - { + public V compute(K key, BiFunction remappingFunction) { + for (; ; ) { V oldVal = map.get(key); - if (oldVal == null) - { - try - { + if (oldVal == null) { + try { semaphore.acquire(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); return null; } V newVal = remappingFunction.apply(key, null); - if (newVal == null) - { + if (newVal == null) { semaphore.release(); return null; } V existing = map.putIfAbsent(key, newVal); - if (existing == null) - { + if (existing == null) { return newVal; - } else - { + } else { semaphore.release(); continue; } - } else - { + } else { return map.compute(key, remappingFunction); } } } - public Set keySet() - { + public Set keySet() { return map.keySet(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java index 3c92c3a..fd3c872 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DataTransform.java @@ -17,61 +17,51 @@ * License along with Pixels. If not, see * . */ - - package io.pixelsdb.pixels.sink.util; - - import com.google.protobuf.ByteString; - import io.pixelsdb.pixels.retina.RetinaProto; - import io.pixelsdb.pixels.sink.SinkProto; - import java.util.ArrayList; - - import java.nio.ByteBuffer; - import java.util.List; - - public class DataTransform - { - private static ByteString longToByteString(long value) - { - byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); - return ByteString.copyFrom(bytes); - } - - @Deprecated - public static void updateTimeStamp(List updateData, long txStartTime) - { - ByteString timestampBytes = longToByteString(txStartTime); - - for (RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder : updateData) - { - int insertDataCount = tableUpdateDataBuilder.getInsertDataCount(); - for (int i = 0; i < insertDataCount; i++) - { - RetinaProto.InsertData.Builder insertBuilder = tableUpdateDataBuilder.getInsertDataBuilder(i); - int colValueCount = insertBuilder.getColValuesCount(); - if (colValueCount > 0) - { - insertBuilder.setColValues(colValueCount - 1, timestampBytes); - } - } - - int updateDataCount = tableUpdateDataBuilder.getUpdateDataCount(); - for (int i = 0; i < updateDataCount; i++) - { - RetinaProto.UpdateData.Builder updateBuilder = tableUpdateDataBuilder.getUpdateDataBuilder(i); - - int colValueCount = updateBuilder.getColValuesCount(); - if (colValueCount > 0) - { - updateBuilder.setColValues(colValueCount - 1, timestampBytes); - } - } - } - } - - - - public static List updateRecordTimestamp(List records, long timestamp) - { + +package io.pixelsdb.pixels.sink.util; + +import com.google.protobuf.ByteString; +import io.pixelsdb.pixels.retina.RetinaProto; +import io.pixelsdb.pixels.sink.SinkProto; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public class DataTransform { + private static ByteString longToByteString(long value) { + byte[] bytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); + return ByteString.copyFrom(bytes); + } + + @Deprecated + public static void updateTimeStamp(List updateData, long txStartTime) { + ByteString timestampBytes = longToByteString(txStartTime); + + for (RetinaProto.TableUpdateData.Builder tableUpdateDataBuilder : updateData) { + int insertDataCount = tableUpdateDataBuilder.getInsertDataCount(); + for (int i = 0; i < insertDataCount; i++) { + RetinaProto.InsertData.Builder insertBuilder = tableUpdateDataBuilder.getInsertDataBuilder(i); + int colValueCount = insertBuilder.getColValuesCount(); + if (colValueCount > 0) { + insertBuilder.setColValues(colValueCount - 1, timestampBytes); + } + } + + int updateDataCount = tableUpdateDataBuilder.getUpdateDataCount(); + for (int i = 0; i < updateDataCount; i++) { + RetinaProto.UpdateData.Builder updateBuilder = tableUpdateDataBuilder.getUpdateDataBuilder(i); + + int colValueCount = updateBuilder.getColValuesCount(); + if (colValueCount > 0) { + updateBuilder.setColValues(colValueCount - 1, timestampBytes); + } + } + } + } + + + public static List updateRecordTimestamp(List records, long timestamp) { if (records == null || records.isEmpty()) { return records; } @@ -84,71 +74,65 @@ public static List updateRecordTimestamp(List 0) { - afterBuilder.setValues(colCount - 1, getTimestampColumn(timestamp)); - } - } - break; - case DELETE: - default: - break; - } - } - - private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord.Builder recordBuilder, SinkProto.ColumnValue timestampColumn) - { - switch (recordBuilder.getOp()) { - case INSERT: - case UPDATE: - case SNAPSHOT: - if (recordBuilder.hasAfter()) { - SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); - int colCount = afterBuilder.getValuesCount(); - if (colCount > 0) { - afterBuilder.setValues(colCount - 1, timestampColumn); - } - } - break; - case DELETE: - default: - break; - } - return recordBuilder.build(); - } - - private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord record, SinkProto.ColumnValue timestampColumn) - { - SinkProto.RowRecord.Builder recordBuilder = record.toBuilder(); - return updateRecordTimestamp(recordBuilder, timestampColumn); - } - - public static String extractTableName(String topic) - { - String[] parts = topic.split("\\."); - return parts[parts.length - 1]; - } - } + private static SinkProto.ColumnValue getTimestampColumn(long timestamp) { + ByteString timestampBytes = longToByteString(timestamp); + return SinkProto.ColumnValue.newBuilder().setValue(timestampBytes).build(); + } + + public static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord record, long timestamp) { + if (record == null) { + return null; + } + SinkProto.ColumnValue timestampColumn = getTimestampColumn(timestamp); + return updateRecordTimestamp(record, timestampColumn); + } + + public static void updateRecordTimestamp(SinkProto.RowRecord.Builder recordBuilder, long timestamp) { + switch (recordBuilder.getOp()) { + case INSERT: + case UPDATE: + case SNAPSHOT: + if (recordBuilder.hasAfter()) { + SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); + int colCount = afterBuilder.getValuesCount(); + if (colCount > 0) { + afterBuilder.setValues(colCount - 1, getTimestampColumn(timestamp)); + } + } + break; + case DELETE: + default: + break; + } + } + + private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord.Builder recordBuilder, SinkProto.ColumnValue timestampColumn) { + switch (recordBuilder.getOp()) { + case INSERT: + case UPDATE: + case SNAPSHOT: + if (recordBuilder.hasAfter()) { + SinkProto.RowValue.Builder afterBuilder = recordBuilder.getAfterBuilder(); + int colCount = afterBuilder.getValuesCount(); + if (colCount > 0) { + afterBuilder.setValues(colCount - 1, timestampColumn); + } + } + break; + case DELETE: + default: + break; + } + return recordBuilder.build(); + } + + private static SinkProto.RowRecord updateRecordTimestamp(SinkProto.RowRecord record, SinkProto.ColumnValue timestampColumn) { + SinkProto.RowRecord.Builder recordBuilder = record.toBuilder(); + return updateRecordTimestamp(recordBuilder, timestampColumn); + } + + public static String extractTableName(String topic) { + String[] parts = topic.split("\\."); + return parts[parts.length - 1]; + } +} diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java b/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java index 2fadbcb..41b2d6f 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/DateUtil.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; @@ -38,11 +38,9 @@ * @author: AntiO2 * @date: 2025/8/21 17:31 */ -public class DateUtil -{ +public class DateUtil { - public static Date fromDebeziumDate(int epochDay) - { + public static Date fromDebeziumDate(int epochDay) { Calendar cal = Calendar.getInstance(); cal.clear(); cal.set(1970, Calendar.JANUARY, 1); // epoch 起点 @@ -51,39 +49,33 @@ public static Date fromDebeziumDate(int epochDay) } // TIMESTAMP(1), TIMESTAMP(2), TIMESTAMP(3) - public static Date fromDebeziumTimestamp(long epochTs) - { + public static Date fromDebeziumTimestamp(long epochTs) { return new Date(epochTs / 1000); } - public static String convertDateToDayString(Date date) - { + public static String convertDateToDayString(Date date) { // "yyyy-MM-dd HH:mm:ss.SSS" DateFormat df = new SimpleDateFormat("yyyy-MM-dd"); String dateToString = df.format(date); return (dateToString); } - public static String convertDateToString(Date date) - { + public static String convertDateToString(Date date) { // "yyyy-MM-dd HH:mm:ss.SSS" DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); String dateToString = df.format(date); return (dateToString); } - public static String convertTimestampToString(Date date) - { - if (date == null) - { + public static String convertTimestampToString(Date date) { + if (date == null) { return null; } SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss.SSS"); return sdf.format(date); } - public static String convertDebeziumTimestampToString(long epochTs) - { + public static String convertDebeziumTimestampToString(long epochTs) { LocalDateTime dateTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(epochTs), ZoneId.systemDefault()); DateTimeFormatter formatter = DatetimeUtils.SQL_LOCAL_DATE_TIME; return dateTime.format(formatter); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java index fbafa43..999fa3e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/EtcdFileRegistry.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; @@ -41,8 +41,7 @@ * @author: AntiO2 * @date: 2025/10/5 08:24 */ -public class EtcdFileRegistry -{ +public class EtcdFileRegistry { private static final Logger LOGGER = LoggerFactory.getLogger(EtcdFileRegistry.class); private static final String REGISTRY_PREFIX = "/sink/proto/registry/"; @@ -53,62 +52,50 @@ public class EtcdFileRegistry private final AtomicInteger nextFileId = new AtomicInteger(0); private String currentFileKey; - public EtcdFileRegistry(String topic, String baseDir) - { + public EtcdFileRegistry(String topic, String baseDir) { this.topic = topic; this.baseDir = baseDir; initRegistry(); } - public static String extractPath(String etcdValue) - { - try - { + public static String extractPath(String etcdValue) { + try { Map meta = OBJECT_MAPPER.readValue(etcdValue, Map.class); return (String) meta.get("path"); - } catch (IOException e) - { + } catch (IOException e) { LOGGER.error("Failed to parse etcd value: {}", etcdValue, e); return null; } } - private void initRegistry() - { + private void initRegistry() { List files = etcd.getKeyValuesByPrefix(filePrefix()); - if (!files.isEmpty()) - { + if (!files.isEmpty()) { int maxId = files.stream() .mapToInt(kv -> extractFileId(kv.getKey().toString())) .max() .orElse(0); nextFileId.set(maxId + 1); LOGGER.info("Initialized registry for topic {} with nextFileId={}", topic, nextFileId.get()); - } else - { + } else { LOGGER.info("No existing files found for topic {}, starting fresh", topic); } } - private String topicPrefix() - { + private String topicPrefix() { return REGISTRY_PREFIX + topic; } - private String filePrefix() - { + private String filePrefix() { return topicPrefix() + "/files/"; } - private int extractFileId(String key) - { - try - { + private int extractFileId(String key) { + try { String fileName = key.substring(key.lastIndexOf('/') + 1); String id = fileName.replace(".proto", ""); return Integer.parseInt(id); - } catch (Exception e) - { + } catch (Exception e) { return 0; } } @@ -116,8 +103,7 @@ private int extractFileId(String key) /** * Create a new file and register it in etcd. */ - public synchronized String createNewFile() - { + public synchronized String createNewFile() { String fileName = String.format("%05d.proto", nextFileId.getAndIncrement()); String fullPath = baseDir + "/" + topic + "/" + fileName; @@ -128,11 +114,9 @@ public synchronized String createNewFile() currentFileKey = filePrefix() + fileName; String jsonValue = null; - try - { + try { jsonValue = OBJECT_MAPPER.writeValueAsString(fileMeta); - } catch (JsonProcessingException e) - { + } catch (JsonProcessingException e) { throw new RuntimeException(e); } @@ -143,16 +127,14 @@ public synchronized String createNewFile() return fullPath; } - public synchronized String getCurrentFileKey() - { + public synchronized String getCurrentFileKey() { return currentFileKey; } /** * List all files (for readers). */ - public List listAllFiles() - { + public List listAllFiles() { List files = etcd.getKeyValuesByPrefix(filePrefix()); return files.stream() .map(kv -> @@ -167,29 +149,25 @@ public List listAllFiles() /** * Mark a file as completed (for writer rotation). */ - public void markFileCompleted(String fileName) - { + public void markFileCompleted(String fileName) { KeyValue kv = etcd.getKeyValue(fileName); if (kv == null) return; Map meta = null; - try - { + try { meta = OBJECT_MAPPER.readValue(kv.getValue().toString(), Map.class); meta.put("completed_at", String.valueOf(System.currentTimeMillis())); meta.put("status", "completed"); String jsonValue = OBJECT_MAPPER.writeValueAsString(meta); etcd.putKeyValue(fileName, jsonValue); - } catch (JsonProcessingException e) - { + } catch (JsonProcessingException e) { throw new RuntimeException(e); } LOGGER.info("Marked file [{}] as completed", fileName); } - public void cleanData() - { + public void cleanData() { etcd.deleteByPrefix(topicPrefix()); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java index 70608e4..8fa0276 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/FlushRateLimiter.java @@ -17,10 +17,9 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; -import com.google.common.util.concurrent.RateLimiter; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import org.slf4j.Logger; @@ -33,19 +32,15 @@ public class FlushRateLimiter { private static final Logger LOGGER = LoggerFactory.getLogger(FlushRateLimiter.class); - + // Configuration derived parameters + private static final long REFRESH_PERIOD_MS = 10; + private static volatile FlushRateLimiter instance; private final Semaphore semaphore; private final boolean enableRateLimiter; private final ScheduledExecutorService scheduler; - - // Configuration derived parameters - private static final long REFRESH_PERIOD_MS = 10; private final int replenishmentAmount; - private static volatile FlushRateLimiter instance; - - private FlushRateLimiter() - { + private FlushRateLimiter() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); int sourceRateLimit = pixelsSinkConfig.getSourceRateLimit(); this.enableRateLimiter = pixelsSinkConfig.isEnableSourceRateLimit(); @@ -62,7 +57,8 @@ private FlushRateLimiter() this.semaphore = new Semaphore(this.replenishmentAmount); - this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> + { Thread t = Executors.defaultThreadFactory().newThread(r); t.setName("Rate-Limiter-Replenish"); t.setDaemon(true); @@ -80,20 +76,10 @@ private FlushRateLimiter() sourceRateLimit, this.replenishmentAmount, REFRESH_PERIOD_MS); } - private void replenishTokens() { - if (semaphore != null) { - semaphore.release(replenishmentAmount); - } - } - - public static FlushRateLimiter getInstance() - { - if(instance == null) - { - synchronized (FlushRateLimiter.class) - { - if(instance == null) - { + public static FlushRateLimiter getInstance() { + if (instance == null) { + synchronized (FlushRateLimiter.class) { + if (instance == null) { instance = new FlushRateLimiter(); } } @@ -101,13 +87,17 @@ public static FlushRateLimiter getInstance() return instance; } - public static FlushRateLimiter getNewInstance() - { + public static FlushRateLimiter getNewInstance() { return new FlushRateLimiter(); } - public void acquire(int num) - { + private void replenishTokens() { + if (semaphore != null) { + semaphore.release(replenishmentAmount); + } + } + + public void acquire(int num) { if (enableRateLimiter && semaphore != null) { try { semaphore.acquire(num); diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java b/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java index 7bb9ce8..a89d571 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/LatencySimulator.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; @@ -26,18 +26,15 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -public class LatencySimulator -{ +public class LatencySimulator { private static final Random RANDOM = ThreadLocalRandom.current(); private static final double longTailProb = 0.05; private static final double longTailScale = 30; private static final double tailVariance = 0.1; private static final double normalVariance = 0.4; - private static long generateLongTailDelay(long baseDelayMs) - { - if (RANDOM.nextDouble() < longTailProb) - { + private static long generateLongTailDelay(long baseDelayMs) { + if (RANDOM.nextDouble() < longTailProb) { double variance = 1 + (RANDOM.nextDouble() * 2 - 1) * tailVariance; return (long) (baseDelayMs * longTailScale * variance); } @@ -45,13 +42,10 @@ private static long generateLongTailDelay(long baseDelayMs) return (long) (baseDelayMs * (1 + (RANDOM.nextDouble() - 0.5) * normalVariance)); } - public static void smartDelay() - { - try - { + public static void smartDelay() { + try { TimeUnit.MILLISECONDS.sleep(generateLongTailDelay(PixelsSinkConfigFactory.getInstance().getMockRpcDelay())); - } catch (InterruptedException ignored) - { + } catch (InterruptedException ignored) { } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java index a3ea787..9001433 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/MetricsFacade.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; import com.google.protobuf.ByteString; @@ -44,8 +44,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -public class MetricsFacade -{ +public class MetricsFacade { private static final Logger LOGGER = LoggerFactory.getLogger(MetricsFacade.class); private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); private static MetricsFacade instance; @@ -82,118 +81,115 @@ public class MetricsFacade private final String freshnessReportPath; private final AtomicBoolean running = new AtomicBoolean(false); - - @Setter - private SinkContextManager sinkContextManager; - private final Thread reportThread; private final Thread freshnessThread; + @Setter + private SinkContextManager sinkContextManager; private long lastRowChangeCount = 0; private long lastTransactionCount = 0; private long lastDebeziumCount = 0; private long lastSerdRowRecordCount = 0; private long lastSerdTxRecordCount = 0; - private MetricsFacade(boolean enabled) - { + private MetricsFacade(boolean enabled) { this.enabled = enabled; - this.debeziumEventCounter = Counter.build() - .name("debezium_event_total") - .help("Debezium Event Total") - .register(); - - this.rowEventCounter = Counter.build() - .name("row_event_total") - .help("Debezium Row Event Total") - .register(); - - this.serdRowRecordCounter = Counter.build() - .name("serd_row_record") - .help("Serialized Row Record Total") - .register(); - - this.serdTxRecordCounter = Counter.build() - .name("serd_tx_record") - .help("Serialized Transaction Record Total") - .register(); - - this.tableChangeCounter = Counter.build() - .name("sink_table_changes_total") - .help("Total processed table changes") - .labelNames("table") - .register(); - - this.rowChangeCounter = Counter.build() - .name("sink_row_changes_total") - .help("Total processed row changes") - .labelNames("table", "operation") - .register(); - - this.transactionCounter = Counter.build() - .name("sink_transactions_total") - .help("Total committed transactions") - .register(); - - this.processingLatency = Summary.build() - .name("sink_processing_latency_seconds") - .help("End-to-end processing latency") - .quantile(0.5, 0.05) - .quantile(0.75, 0.01) - .quantile(0.95, 0.005) - .quantile(0.99, 0.001) - .register(); - - this.rawDataThroughputCounter = Counter.build() - .name("sink_data_throughput_counter") - .help("Data throughput") - .register(); - - this.transServiceLatency = Summary.build() - .name("trans_service_latency_seconds") - .help("End-to-end processing latency") - .quantile(0.5, 0.05) - .quantile(0.75, 0.01) - .quantile(0.95, 0.005) - .quantile(0.99, 0.001) - .register(); - - this.indexServiceLatency = Summary.build() - .name("index_service_latency_seconds") - .help("End-to-end processing latency") - .quantile(0.5, 0.05) - .quantile(0.75, 0.01) - .quantile(0.95, 0.005) - .quantile(0.99, 0.001) - .register(); - - this.retinaServiceLatency = Summary.build() - .name("retina_service_latency_seconds") - .help("End-to-end processing latency") - .quantile(0.5, 0.05) - .quantile(0.75, 0.01) - .quantile(0.95, 0.005) - .quantile(0.99, 0.001) - .register(); - - this.writerLatency = Summary.build() - .name("write_latency_seconds") - .help("Write latency") - .labelNames("table") - .quantile(0.5, 0.05) - .quantile(0.75, 0.01) - .quantile(0.95, 0.005) - .quantile(0.99, 0.001) - .register(); - - this.totalLatency = Summary.build() - .name("total_latency_seconds") - .help("total latency to ETL a row change event") - .labelNames("table", "operation") - .quantile(0.5, 0.05) - .quantile(0.75, 0.01) - .quantile(0.95, 0.005) - .quantile(0.99, 0.001) - .register(); + this.debeziumEventCounter = Counter.build() + .name("debezium_event_total") + .help("Debezium Event Total") + .register(); + + this.rowEventCounter = Counter.build() + .name("row_event_total") + .help("Debezium Row Event Total") + .register(); + + this.serdRowRecordCounter = Counter.build() + .name("serd_row_record") + .help("Serialized Row Record Total") + .register(); + + this.serdTxRecordCounter = Counter.build() + .name("serd_tx_record") + .help("Serialized Transaction Record Total") + .register(); + + this.tableChangeCounter = Counter.build() + .name("sink_table_changes_total") + .help("Total processed table changes") + .labelNames("table") + .register(); + + this.rowChangeCounter = Counter.build() + .name("sink_row_changes_total") + .help("Total processed row changes") + .labelNames("table", "operation") + .register(); + + this.transactionCounter = Counter.build() + .name("sink_transactions_total") + .help("Total committed transactions") + .register(); + + this.processingLatency = Summary.build() + .name("sink_processing_latency_seconds") + .help("End-to-end processing latency") + .quantile(0.5, 0.05) + .quantile(0.75, 0.01) + .quantile(0.95, 0.005) + .quantile(0.99, 0.001) + .register(); + + this.rawDataThroughputCounter = Counter.build() + .name("sink_data_throughput_counter") + .help("Data throughput") + .register(); + + this.transServiceLatency = Summary.build() + .name("trans_service_latency_seconds") + .help("End-to-end processing latency") + .quantile(0.5, 0.05) + .quantile(0.75, 0.01) + .quantile(0.95, 0.005) + .quantile(0.99, 0.001) + .register(); + + this.indexServiceLatency = Summary.build() + .name("index_service_latency_seconds") + .help("End-to-end processing latency") + .quantile(0.5, 0.05) + .quantile(0.75, 0.01) + .quantile(0.95, 0.005) + .quantile(0.99, 0.001) + .register(); + + this.retinaServiceLatency = Summary.build() + .name("retina_service_latency_seconds") + .help("End-to-end processing latency") + .quantile(0.5, 0.05) + .quantile(0.75, 0.01) + .quantile(0.95, 0.005) + .quantile(0.99, 0.001) + .register(); + + this.writerLatency = Summary.build() + .name("write_latency_seconds") + .help("Write latency") + .labelNames("table") + .quantile(0.5, 0.05) + .quantile(0.75, 0.01) + .quantile(0.95, 0.005) + .quantile(0.99, 0.001) + .register(); + + this.totalLatency = Summary.build() + .name("total_latency_seconds") + .help("total latency to ETL a row change event") + .labelNames("table", "operation") + .quantile(0.5, 0.05) + .quantile(0.75, 0.01) + .quantile(0.95, 0.005) + .quantile(0.99, 0.001) + .register(); this.tableFreshness = Summary.build() .name("data_freshness_latency_ms") @@ -204,29 +200,27 @@ private MetricsFacade(boolean enabled) .quantile(0.99, 0.001) .register(); - this.transactionRowCountHistogram = Histogram.build() - .name("transaction_row_count_histogram") - .help("Distribution of row counts within a single transaction") - .buckets(1, 5, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100, 200) - .register(); - this.primaryKeyUpdateDistribution = Histogram.build() - .name("primary_key_update_distribution") - .help("Distribution of primary key updates by logical bucket/hash for hot spot analysis") - .labelNames("table") // Table name tag - .buckets(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) // 10 buckets for distribution - .register(); - this.freshness = new SynchronizedDescriptiveStatistics(); - this.rowChangeSpeed = new SynchronizedDescriptiveStatistics(); + this.transactionRowCountHistogram = Histogram.build() + .name("transaction_row_count_histogram") + .help("Distribution of row counts within a single transaction") + .buckets(1, 5, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100, 200) + .register(); + this.primaryKeyUpdateDistribution = Histogram.build() + .name("primary_key_update_distribution") + .help("Distribution of primary key updates by logical bucket/hash for hot spot analysis") + .labelNames("table") // Table name tag + .buckets(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) // 10 buckets for distribution + .register(); + this.freshness = new SynchronizedDescriptiveStatistics(); + this.rowChangeSpeed = new SynchronizedDescriptiveStatistics(); freshnessReportInterval = config.getFreshnessReportInterval(); freshnessReportPath = config.getMonitorFreshnessReportFile(); freshnessAvg = new OneSecondAverage(freshnessReportInterval); freshnessVerbose = config.isSinkMonitorFreshnessVerbose(); - if(freshnessVerbose) - { + if (freshnessVerbose) { freshnessHistory = new FreshnessHistory(); - } else - { + } else { freshnessHistory = null; } @@ -234,183 +228,145 @@ private MetricsFacade(boolean enabled) monitorReportEnabled = config.isMonitorReportEnabled(); monitorReportInterval = config.getMonitorReportInterval(); monitorReportPath = config.getMonitorReportFile(); - if (monitorReportEnabled) - { + if (monitorReportEnabled) { running.set(true); reportThread = new Thread(this::run, "Metrics Report Thread"); LOGGER.info("Metrics Report Thread Started"); reportThread.start(); freshnessThread = new Thread(this::runFreshness, "Freshness Thread"); freshnessThread.start(); - } else - { + } else { reportThread = null; freshnessThread = null; } } - private static synchronized void initialize() - { - if (instance == null) - { + private static synchronized void initialize() { + if (instance == null) { instance = new MetricsFacade(config.isMonitorEnabled()); LOGGER.info("Init Metrics Facade"); } } - public static MetricsFacade getInstance() - { - if (instance == null) - { + public static MetricsFacade getInstance() { + if (instance == null) { initialize(); } return instance; } - public void stop() - { + public void stop() { running.set(false); - if (reportThread != null) - { + if (reportThread != null) { reportThread.interrupt(); } - if (freshnessThread != null) - { + if (freshnessThread != null) { freshnessThread.interrupt(); } LOGGER.info("Monitor report thread stopped."); } - public void recordDebeziumEvent() - { - if (enabled && debeziumEventCounter != null) - { + public void recordDebeziumEvent() { + if (enabled && debeziumEventCounter != null) { debeziumEventCounter.inc(); } } - public void recordRowChange(String table, SinkProto.OperationType operation) - { + public void recordRowChange(String table, SinkProto.OperationType operation) { recordRowChange(table, operation, 1); } - public void recordRowChange(String table, SinkProto.OperationType operation, int rows) - { - if (enabled && rowChangeCounter != null) - { + public void recordRowChange(String table, SinkProto.OperationType operation, int rows) { + if (enabled && rowChangeCounter != null) { tableChangeCounter.labels(table).inc(rows); rowChangeCounter.labels(table, operation.toString()).inc(rows); } } - public void recordSerdRowChange() - { + public void recordSerdRowChange() { recordSerdRowChange(1); } - public void recordSerdRowChange(int i) - { - if (enabled && serdRowRecordCounter != null) - { + public void recordSerdRowChange(int i) { + if (enabled && serdRowRecordCounter != null) { serdRowRecordCounter.inc(i); } } - public void recordSerdTxChange() - { + public void recordSerdTxChange() { recordSerdTxChange(1); } - public void recordSerdTxChange(int i) - { - if (enabled && serdTxRecordCounter != null) - { + public void recordSerdTxChange(int i) { + if (enabled && serdTxRecordCounter != null) { serdTxRecordCounter.inc(i); } } - public void recordTransaction(int i) - { - if (enabled && transactionCounter != null) - { + public void recordTransaction(int i) { + if (enabled && transactionCounter != null) { transactionCounter.inc(i); } } - public void recordTransaction() - { + public void recordTransaction() { recordTransaction(1); } - public Summary.Timer startProcessLatencyTimer() - { + public Summary.Timer startProcessLatencyTimer() { return enabled ? processingLatency.startTimer() : null; } - public Summary.Timer startIndexLatencyTimer() - { + public Summary.Timer startIndexLatencyTimer() { return enabled ? indexServiceLatency.startTimer() : null; } - public Summary.Timer startTransLatencyTimer() - { + public Summary.Timer startTransLatencyTimer() { return enabled ? transServiceLatency.startTimer() : null; } - public Summary.Timer startRetinaLatencyTimer() - { + public Summary.Timer startRetinaLatencyTimer() { return enabled ? retinaServiceLatency.startTimer() : null; } - public Summary.Timer startWriteLatencyTimer(String tableName) - { + public Summary.Timer startWriteLatencyTimer(String tableName) { return enabled ? writerLatency.labels(tableName).startTimer() : null; } - public void addRawData(double data) - { + public void addRawData(double data) { rawDataThroughputCounter.inc(data); } - public void recordTotalLatency(RowChangeEvent event) - { - if (event.getTimeStamp() != 0) - { + public void recordTotalLatency(RowChangeEvent event) { + if (event.getTimeStamp() != 0) { long recordLatency = System.currentTimeMillis() - event.getTimeStamp(); totalLatency.labels(event.getFullTableName(), event.getOp().toString()).observe(recordLatency); } } - public void recordRowEvent() - { + public void recordRowEvent() { recordRowEvent(1); } - public void recordRowEvent(int i) - { - if (enabled && rowEventCounter != null) - { + public void recordRowEvent(int i) { + if (enabled && rowEventCounter != null) { rowEventCounter.inc(i); } } - public int getRecordRowEvent() - { + public int getRecordRowEvent() { return (int) rowEventCounter.get(); } - public int getTransactionEvent() - { + public int getTransactionEvent() { return (int) transactionCounter.get(); } - public void recordTableFreshness(String table, double freshnessMill) - { - if(!enabled) - { + public void recordTableFreshness(String table, double freshnessMill) { + if (!enabled) { return; } @@ -418,20 +374,16 @@ public void recordTableFreshness(String table, double freshnessMill) recordFreshness(freshnessMill); } - public void recordFreshness(double freshnessMill) - { - if(enabled && freshness != null) - { + public void recordFreshness(double freshnessMill) { + if (enabled && freshness != null) { freshness.addValue(freshnessMill); } - if(freshnessAvg != null) - { + if (freshnessAvg != null) { freshnessAvg.record(freshnessMill); } - if(freshnessVerbose) - { + if (freshnessVerbose) { freshnessHistory.record(freshnessMill); } } @@ -471,89 +423,68 @@ public void recordPrimaryKeyUpdateDistribution(String table, ByteString pkValue) LOGGER.debug("Table {}: PK {} mapped to bucket index {}", table, numericPK, bucketIndex); } - public void recordTransactionRowCount(int rowCount) - { - if (enabled && transactionRowCountHistogram != null) - { + + public void recordTransactionRowCount(int rowCount) { + if (enabled && transactionRowCountHistogram != null) { // Use observe() to add the value to the Histogram's configured buckets. transactionRowCountHistogram.observe(rowCount); } } - public void run() - { - while (running.get()) - { - try - { + public void run() { + while (running.get()) { + try { Thread.sleep(monitorReportInterval); logPerformance(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; - } catch (Throwable t) - { + } catch (Throwable t) { LOGGER.warn("Error while reporting performance.", t); } } } - public void runFreshness() - { - try - { + public void runFreshness() { + try { Thread.sleep(monitorReportInterval); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - while (running.get()) - { - try - { + while (running.get()) { + try { Thread.sleep(freshnessReportInterval); - try (FileWriter fw = new FileWriter(freshnessReportPath, true)) - { - if(freshnessVerbose) - { + try (FileWriter fw = new FileWriter(freshnessReportPath, true)) { + if (freshnessVerbose) { List detailedRecords = freshnessHistory.pollAll(); - if (!detailedRecords.isEmpty()) - { - for (FreshnessHistory.Record record : detailedRecords) - { + if (!detailedRecords.isEmpty()) { + for (FreshnessHistory.Record record : detailedRecords) { fw.write(record.toString() + "\n"); } fw.flush(); } - } else - { + } else { long now = System.currentTimeMillis(); double avg = freshnessAvg.getWindowAverage(); - if(Double.isNaN(avg)) - { + if (Double.isNaN(avg)) { continue; } fw.write(now + "," + avg + "\n"); fw.flush(); } - } catch (IOException e) - { + } catch (IOException e) { LOGGER.warn("Failed to write perf metrics: " + e.getMessage()); } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; - } catch (Throwable t) - { + } catch (Throwable t) { LOGGER.warn("Error while reporting performance.", t); } } } - public void logPerformance() - { + public void logPerformance() { long currentRows = (long) rowEventCounter.get(); long currentTxns = (long) transactionCounter.get(); long currentDebezium = (long) debeziumEventCounter.get(); @@ -626,12 +557,10 @@ public void logPerformance() String time = LocalDateTime.now().format(DateTimeFormatter.ofPattern("HH:mm:ss")); // Append to CSV for plotting - try (FileWriter fw = new FileWriter(monitorReportPath, true)) - { + try (FileWriter fw = new FileWriter(monitorReportPath, true)) { fw.write(String.format("%s,%.2f,%.2f,%.2f,%.2f,%.2f%n", time, rowOips, txnOips, dbOips, serdRowsOips, serdTxsOips)); - } catch (IOException e) - { + } catch (IOException e) { LOGGER.warn("Failed to write perf metrics: " + e.getMessage()); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java b/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java index 5652c64..2adac60 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java +++ b/src/main/java/io/pixelsdb/pixels/sink/util/TableCounters.java @@ -17,41 +17,35 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.util; /** * Inner class to hold and manage per-table transaction row counts. */ -public class TableCounters -{ +public class TableCounters { + private final int totalCount; // The expected total number of rows // currentCount is volatile for visibility across threads, as it's incremented during writeRow. private volatile int currentCount = 0; - private final int totalCount; // The expected total number of rows - public TableCounters(int totalCount) - { + public TableCounters(int totalCount) { this.totalCount = totalCount; } - public void increment() - { + public void increment() { currentCount++; } - public boolean isComplete() - { + public boolean isComplete() { // Checks if the processed count meets or exceeds the expected total count. return currentCount >= totalCount; } - public int getCurrentCount() - { + public int getCurrentCount() { return currentCount; } - public int getTotalCount() - { + public int getTotalCount() { return totalCount; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java index 1ad46b6..6a49495 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/AbstractBucketedWriter.java @@ -24,27 +24,19 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; -public abstract class AbstractBucketedWriter -{ - public void writeRowChangeEvent(RowChangeEvent event, C context) throws SinkException - { - if (event == null) - { +public abstract class AbstractBucketedWriter { + public void writeRowChangeEvent(RowChangeEvent event, C context) throws SinkException { + if (event == null) { return; } event.initIndexKey(); - switch (event.getOp()) - { - case UPDATE -> - { - if (!event.isPkChanged()) - { + switch (event.getOp()) { + case UPDATE -> { + if (!event.isPkChanged()) { emitBefore(event, context); - } - else - { + } else { emitPkChangedUpdate(event, context); } } @@ -53,8 +45,7 @@ public void writeRowChangeEvent(RowChangeEvent event, C context) throws SinkExce case INSERT, SNAPSHOT -> emitAfter(event, context); - case UNRECOGNIZED -> - { + case UNRECOGNIZED -> { return; } } @@ -62,20 +53,17 @@ public void writeRowChangeEvent(RowChangeEvent event, C context) throws SinkExce /* ================= hook points ================= */ - protected void emitBefore(RowChangeEvent event, C context) - { + protected void emitBefore(RowChangeEvent event, C context) { int bucketId = event.getBeforeBucketFromIndex(); emit(event, bucketId, context); } - protected void emitAfter(RowChangeEvent event, C context) - { + protected void emitAfter(RowChangeEvent event, C context) { int bucketId = event.getAfterBucketFromIndex(); emit(event, bucketId, context); } - protected void emitPkChangedUpdate(RowChangeEvent event, C context) throws SinkException - { + protected void emitPkChangedUpdate(RowChangeEvent event, C context) throws SinkException { // DELETE (before) RowChangeEvent deleteEvent = buildDeleteEvent(event); emitBefore(deleteEvent, context); @@ -89,8 +77,7 @@ protected void emitPkChangedUpdate(RowChangeEvent event, C context) throws SinkE /* ================= helpers ================= */ - private RowChangeEvent buildDeleteEvent(RowChangeEvent event) throws SinkException - { + private RowChangeEvent buildDeleteEvent(RowChangeEvent event) throws SinkException { SinkProto.RowRecord.Builder builder = event.getRowRecord().toBuilder() .clearAfter() @@ -102,8 +89,7 @@ private RowChangeEvent buildDeleteEvent(RowChangeEvent event) throws SinkExcepti return deleteEvent; } - private RowChangeEvent buildInsertEvent(RowChangeEvent event) throws SinkException - { + private RowChangeEvent buildInsertEvent(RowChangeEvent event) throws SinkException { SinkProto.RowRecord.Builder builder = event.getRowRecord().toBuilder() .clearBefore() diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java index 41c73c0..ccf5b60 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/NoneWriter.java @@ -17,32 +17,27 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.exception.SinkException; import io.pixelsdb.pixels.sink.util.MetricsFacade; -import io.pixelsdb.pixels.sink.util.TableCounters; import io.pixelsdb.pixels.sink.writer.retina.SinkContext; -import lombok.Getter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.ReentrantLock; /** * NoneWriter implementation used for testing and metrics collection. * It tracks transaction completeness based on row counts provided in the TXEND metadata, * ensuring robust handling of out-of-order and concurrent TX BEGIN, TX END, and ROWChange events. */ -public class NoneWriter implements PixelsSinkWriter -{ +public class NoneWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(NoneWriter.class); private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); @@ -59,20 +54,17 @@ public class NoneWriter implements PixelsSinkWriter * * @param transId The ID of the transaction to check. */ - private void checkAndCleanupTransaction(String transId) - { + private void checkAndCleanupTransaction(String transId) { TransactionContext context = transTracker.get(transId); - if (context == null) - { + if (context == null) { return; } boolean allComplete = context.sinkContext.isCompleted(); int actualProcessedRows = context.sinkContext.getProcessedRowsNum(); - if (allComplete) - { + if (allComplete) { // All rows expected have been processed. Remove and record metrics. transTracker.remove(transId); LOGGER.trace("Transaction {} successfully completed and removed from tracker. Total rows: {}.", transId, actualProcessedRows); @@ -80,86 +72,76 @@ private void checkAndCleanupTransaction(String transId) // Record final transaction metrics only upon completion metricsFacade.recordTransaction(); metricsFacade.recordTransactionRowCount(actualProcessedRows); - } else - { + } else { // Not complete, keep tracking LOGGER.debug("Transaction {} is partially complete ({} rows processed). Keeping tracker entry.", transId, actualProcessedRows); } } @Override - public void flush() - { + public void flush() { // No-op for NoneWriter } // --- Interface Methods --- @Override - public boolean writeRow(RowChangeEvent rowChangeEvent) - { - metricsFacade.recordRowEvent(); - metricsFacade.recordRowChange(rowChangeEvent.getTable(), rowChangeEvent.getOp()); - try - { - rowChangeEvent.initIndexKey(); - metricsFacade.recordPrimaryKeyUpdateDistribution(rowChangeEvent.getTable(), rowChangeEvent.getAfterKey().getKey()); - - // Get transaction ID and table name - String transId = rowChangeEvent.getTransaction().getId(); - String fullTable = rowChangeEvent.getFullTableName(); - - // 1. Get or create the transaction context - TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); - - context.sinkContext.getTableCounterLock().lock(); - context.incrementEndCount(fullTable); - checkAndCleanupTransaction(transId); - context.sinkContext.getTableCounterLock().unlock(); - } catch (SinkException e) - { - throw new RuntimeException("Error processing row key or metrics.", e); - } - return true; + public boolean writeRow(RowChangeEvent rowChangeEvent) { + metricsFacade.recordRowEvent(); + metricsFacade.recordRowChange(rowChangeEvent.getTable(), rowChangeEvent.getOp()); + try { + rowChangeEvent.initIndexKey(); + metricsFacade.recordPrimaryKeyUpdateDistribution(rowChangeEvent.getTable(), rowChangeEvent.getAfterKey().getKey()); + + // Get transaction ID and table name + String transId = rowChangeEvent.getTransaction().getId(); + String fullTable = rowChangeEvent.getFullTableName(); + + // 1. Get or create the transaction context + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); + + context.sinkContext.getTableCounterLock().lock(); + context.incrementEndCount(fullTable); + checkAndCleanupTransaction(transId); + context.sinkContext.getTableCounterLock().unlock(); + } catch (SinkException e) { + throw new RuntimeException("Error processing row key or metrics.", e); + } + return true; } @Override - public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) - { - String transId = transactionMetadata.getId(); - - if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) - { - // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). - transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); - LOGGER.debug("Transaction {} BEGIN received.", transId); - - } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) - { - // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. - - // Get existing context or create a new one (in case BEGIN was missed). - TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); - context.sinkContext.getTableCounterLock().lock(); - context.sinkContext.setEndTx(transactionMetadata); - checkAndCleanupTransaction(transId); - context.sinkContext.getTableCounterLock().unlock(); - } - return true; + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { + String transId = transactionMetadata.getId(); + + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) { + // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). + transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); + LOGGER.debug("Transaction {} BEGIN received.", transId); + + } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) { + // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. + + // Get existing context or create a new one (in case BEGIN was missed). + TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext(transId)); + context.sinkContext.getTableCounterLock().lock(); + context.sinkContext.setEndTx(transactionMetadata); + checkAndCleanupTransaction(transId); + context.sinkContext.getTableCounterLock().unlock(); + } + return true; } @Override - public void close() throws IOException - { + public void close() throws IOException { // No-op for NoneWriter LOGGER.info("Remaining unfinished transactions on close: {}", transTracker.size()); // Log details of transactions that were never completed - if (!transTracker.isEmpty()) - { + if (!transTracker.isEmpty()) { transTracker.forEach((transId, context) -> { - LOGGER.warn("Unfinished transaction {}",transId); + LOGGER.warn("Unfinished transaction {}", transId); }); } } @@ -168,14 +150,12 @@ public void close() throws IOException * Helper class to manage the state of a single transaction, decoupling the row accumulation * from the final TableCounters initialization (which requires total counts from TX END). */ - public static class TransactionContext - { + public static class TransactionContext { // Key: Full Table Name, Value: Row Count private SinkContext sinkContext = null; - TransactionContext(String txId) - { + TransactionContext(String txId) { this.sinkContext = new SinkContext(txId); } @@ -183,8 +163,7 @@ public static class TransactionContext /** * @param table Full table name */ - public void incrementEndCount(String table) - { + public void incrementEndCount(String table) { sinkContext.updateCounter(table, 1); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java index 882e26f..42bea45 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkMode.java @@ -17,24 +17,20 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer; -public enum PixelsSinkMode -{ +public enum PixelsSinkMode { CSV, RETINA, PROTO, FLINK, NONE; - public static PixelsSinkMode fromValue(String value) - { - for (PixelsSinkMode mode : values()) - { - if (mode.name().equalsIgnoreCase(value)) - { + public static PixelsSinkMode fromValue(String value) { + for (PixelsSinkMode mode : values()) { + if (mode.name().equalsIgnoreCase(value)) { return mode; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java index 72ca26b..d933a94 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.SinkProto; @@ -25,8 +25,7 @@ import java.io.Closeable; -public interface PixelsSinkWriter extends Closeable -{ +public interface PixelsSinkWriter extends Closeable { void flush(); boolean writeRow(RowChangeEvent rowChangeEvent); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java index 20cef21..dd884b5 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/PixelsSinkWriterFactory.java @@ -17,37 +17,30 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; import io.pixelsdb.pixels.sink.writer.csv.CsvWriter; +import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; import io.pixelsdb.pixels.sink.writer.proto.ProtoWriter; import io.pixelsdb.pixels.sink.writer.retina.RetinaWriter; -import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; import java.io.IOException; -public class PixelsSinkWriterFactory -{ +public class PixelsSinkWriterFactory { private static final PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); private static volatile PixelsSinkWriter writer = null; - static public PixelsSinkWriter getWriter() - { - if (writer == null) - { - synchronized (PixelsSinkWriterFactory.class) - { - if (writer == null) - { - try - { - switch (config.getPixelsSinkMode()) - { + static public PixelsSinkWriter getWriter() { + if (writer == null) { + synchronized (PixelsSinkWriterFactory.class) { + if (writer == null) { + try { + switch (config.getPixelsSinkMode()) { case CSV: writer = new CsvWriter(); break; @@ -64,8 +57,7 @@ static public PixelsSinkWriter getWriter() writer = new NoneWriter(); break; } - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException("Can't create writer", e); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java index 85edf57..f3baf07 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/csv/CsvWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.csv; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -48,8 +48,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -public class CsvWriter implements PixelsSinkWriter -{ +public class CsvWriter implements PixelsSinkWriter { private static final Logger log = LoggerFactory.getLogger(CsvWriter.class); @Getter private static final PixelsSinkMode pixelsSinkMode = PixelsSinkMode.CSV; @@ -66,14 +65,12 @@ public class CsvWriter implements PixelsSinkWriter private final String CSV_DELIMITER = "|"; private final Long recordCnt = 0L; - public CsvWriter() throws IOException - { + public CsvWriter() throws IOException { this.databaseName = config.getCaptureDatabase(); this.baseOutputPath = Paths.get(config.getCsvSinkPath(), databaseName); this.enableHeader = config.isSinkCsvEnableHeader(); - if (!Files.exists(baseOutputPath)) - { + if (!Files.exists(baseOutputPath)) { Files.createDirectories(baseOutputPath); } @@ -84,100 +81,81 @@ public CsvWriter() throws IOException } @Override - public void flush() - { + public void flush() { writeLock.lock(); - try - { - for (FileChannel channel : tableWriters.values()) - { - try - { + try { + for (FileChannel channel : tableWriters.values()) { + try { channel.force(true); - } catch (IOException e) - { + } catch (IOException e) { log.error("Failed to flush channel {}", channel, e); } } writeCounter.set(0); - } finally - { + } finally { writeLock.unlock(); } } @Override - public boolean writeRow(RowChangeEvent event) - { + public boolean writeRow(RowChangeEvent event) { final String tableName = event.getTable(); - if (event.getOp() == SinkProto.OperationType.DELETE) - { + if (event.getOp() == SinkProto.OperationType.DELETE) { return true; } // Map message = event.getAfterData(); Map message = null; writeLock.lock(); - try - { + try { FileChannel channel = getOrCreateChannel(event); String csvLine = convertToCSV(message); ByteBuffer buffer = ByteBuffer.wrap((csvLine + "\n").getBytes(StandardCharsets.UTF_8)); - while (buffer.hasRemaining()) - { + while (buffer.hasRemaining()) { channel.write(buffer); } - if (writeCounter.incrementAndGet() % PixelsSinkDefaultConfig.SINK_CSV_RECORD_FLUSH == 0) - { + if (writeCounter.incrementAndGet() % PixelsSinkDefaultConfig.SINK_CSV_RECORD_FLUSH == 0) { channel.force(false); } return true; - } catch (IOException e) - { + } catch (IOException e) { log.error("Write failed for table {}: {}", tableName, e.getMessage()); return false; - } finally - { + } finally { writeLock.unlock(); } } @Override - public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) - { + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { // TODO(AntiO2): Write Trans info return false; } - private FileChannel getOrCreateChannel(RowChangeEvent event) throws IOException - { + private FileChannel getOrCreateChannel(RowChangeEvent event) throws IOException { String tableName = event.getTable(); return tableWriters.computeIfAbsent(tableName, key -> { - try - { + try { Path tablePath = baseOutputPath.resolve(tableName + ".csv"); FileChannel channel = FileChannel.open(tablePath, StandardOpenOption.CREATE, StandardOpenOption.APPEND, StandardOpenOption.WRITE); - if (channel.size() == 0) - { + if (channel.size() == 0) { String header = String.join(CSV_DELIMITER, getHeaderFields(event)); channel.write(ByteBuffer.wrap((header + "\n").getBytes())); } return channel; - } catch (IOException e) - { + } catch (IOException e) { throw new UncheckedIOException("Failed to create channel for " + tableName, e); } }); } - private String convertToCSV(Map message) - { + private String convertToCSV(Map message) { return message.values().stream() .map(obj -> { @@ -187,37 +165,29 @@ private String convertToCSV(Map message) .collect(Collectors.joining(CSV_DELIMITER)); } - private List getHeaderFields(RowChangeEvent event) - { + private List getHeaderFields(RowChangeEvent event) { return event.getSchema().getFieldNames(); } @Override - public void close() throws IOException - { + public void close() throws IOException { flushScheduler.shutdown(); - try - { - if (!flushScheduler.awaitTermination(5, TimeUnit.SECONDS)) - { + try { + if (!flushScheduler.awaitTermination(5, TimeUnit.SECONDS)) { flushScheduler.shutdownNow(); } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); } globalLock.lock(); - try - { - for (FileChannel channel : tableWriters.values()) - { + try { + for (FileChannel channel : tableWriters.values()) { channel.close(); } tableWriters.clear(); - } finally - { + } finally { globalLock.unlock(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java index 4a650b1..4fdb4ae 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/FlinkPollingWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.flink; import io.pixelsdb.pixels.common.metadata.SchemaTableName; @@ -50,10 +50,8 @@ public class FlinkPollingWriter extends AbstractBucketedWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(FlinkPollingWriter.class); - record TableBucketKey(SchemaTableName table, int bucketId) {} // Core data structure: A thread-safe map from table name to a thread-safe blocking queue. private final Map> tableQueues; - // The gRPC server instance managed by this writer. private final PollingRpcServer pollingRpcServer; @@ -71,7 +69,7 @@ public FlinkPollingWriter() { int rpcPort = config.getSinkFlinkServerPort(); // 2. Create the gRPC service implementation first, passing a reference to this writer. PixelsPollingServiceImpl service = new PixelsPollingServiceImpl(this); - + // 3. Create the PollingRpcServer instance with the service and port. LOGGER.info("Attempting to start gRPC Polling Server on port {}...", rpcPort); this.pollingRpcServer = new PollingRpcServer(service, rpcPort); @@ -101,13 +99,10 @@ public boolean writeRow(RowChangeEvent event) { return false; } - try - { + try { writeRowChangeEvent(event, null); return true; - } - catch (Exception e) - { + } catch (Exception e) { LOGGER.error( "Failed to process and write row for table: {}", event.getFullTableName(), @@ -122,11 +117,11 @@ public boolean writeRow(RowChangeEvent event) { * Implements long-polling logic: if the queue is empty, it blocks for a specified timeout. * batchSize acts as an upper limit on the number of records pulled to prevent oversized RPC responses. * - * @param tableName The name of the table to pull data from + * @param tableName The name of the table to pull data from * @param bucketId - * @param batchSize The maximum number of records to pull - * @param timeout The maximum time to wait for data - * @param unit The time unit for the timeout + * @param batchSize The maximum number of records to pull + * @param timeout The maximum time to wait for data + * @param unit The time unit for the timeout * @return A list of RowRecords, which will be empty if no data is available before the timeout. * @throws InterruptedException if the thread is interrupted while waiting */ @@ -136,22 +131,19 @@ public List pollRecords( int batchSize, long timeout, TimeUnit unit - ) throws InterruptedException - { + ) throws InterruptedException { List records = new ArrayList<>(batchSize); TableBucketKey key = new TableBucketKey(tableName, bucketId); BlockingQueue queue = tableQueues.get(key); - if (queue == null) - { + if (queue == null) { unit.sleep(timeout); return records; } SinkProto.RowRecord first = queue.poll(timeout, unit); - if (first == null) - { + if (first == null) { return records; } @@ -165,7 +157,6 @@ public List pollRecords( return records; } - /** * This implementation does not involve transactions, so this method is a no-op. */ @@ -186,8 +177,7 @@ public void flush() { * Cleans up resources on close. This is where we stop the gRPC server. */ @Override - public void close() throws IOException - { + public void close() throws IOException { LOGGER.info("Closing FlinkPollingWriter..."); if (this.pollingRpcServer != null) { LOGGER.info("Attempting to shut down the gRPC Polling Server..."); @@ -200,8 +190,7 @@ public void close() throws IOException } @Override - protected void emit(RowChangeEvent event, int bucketId, Void unused) - { + protected void emit(RowChangeEvent event, int bucketId, Void unused) { TableBucketKey key = new TableBucketKey(event.getSchemaTableName(), bucketId); @@ -211,16 +200,13 @@ protected void emit(RowChangeEvent event, int bucketId, Void unused) k -> new LinkedBlockingQueue<>(PixelsSinkConstants.MAX_QUEUE_SIZE) ); - try - { + try { queue.put(event.getRowRecord()); LOGGER.debug( "Enqueued row for table {}, bucket {}, queueSize={}", event.getFullTableName(), bucketId, queue.size() ); - } - catch (InterruptedException e) - { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException( "Interrupted while enqueueing row for " + event.getFullTableName(), @@ -228,4 +214,7 @@ protected void emit(RowChangeEvent event, int bucketId, Void unused) ); } } + + record TableBucketKey(SchemaTableName table, int bucketId) { + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java index 0db65f3..94ed1aa 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/flink/PixelsPollingServiceImpl.java @@ -17,41 +17,30 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.flink; import io.grpc.stub.StreamObserver; import io.pixelsdb.pixels.common.metadata.SchemaTableName; +import io.pixelsdb.pixels.sink.PixelsPollingServiceGrpc; +import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; -import io.pixelsdb.pixels.sink.SinkProto; -import io.pixelsdb.pixels.sink.PixelsPollingServiceGrpc; import io.pixelsdb.pixels.sink.util.FlushRateLimiter; -import io.pixelsdb.pixels.sink.writer.flink.FlinkPollingWriter; -import io.pixelsdb.pixels.sink.util.DataTransform; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.xml.validation.Schema; import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; -/** - * 实现了 .proto 文件中定义的 PixelsPollingService 服务。 - * 它处理来自客户端的 PollRequest,并从 FlinkPollingWriter 中拉取数据进行响应。 - */ -// *** 核心修复: 继承自 gRPC 生成的基类 *** public class PixelsPollingServiceImpl extends PixelsPollingServiceGrpc.PixelsPollingServiceImplBase { private static final Logger LOGGER = LoggerFactory.getLogger(PixelsPollingServiceImpl.class); private final FlinkPollingWriter writer; private final int pollBatchSize; private final long pollTimeoutMs; private final FlushRateLimiter flushRateLimiter; - /** - * 构造函数,注入 FlinkPollingWriter 并初始化服务器端配置。 - * @param writer 数据缓冲区的实例。 - */ + public PixelsPollingServiceImpl(FlinkPollingWriter writer) { if (writer == null) { throw new IllegalArgumentException("FlinkPollingWriter cannot be null."); @@ -73,10 +62,8 @@ public void pollEvents(SinkProto.PollRequest request, StreamObserver records = new ArrayList<>(pollBatchSize); try { - for (int bucketId : request.getBucketsList()) - { - if (records.size() >= pollBatchSize) - { + for (int bucketId : request.getBucketsList()) { + if (records.size() >= pollBatchSize) { break; } @@ -89,15 +76,13 @@ public void pollEvents(SinkProto.PollRequest request, StreamObserver. */ - + package io.pixelsdb.pixels.sink.writer.flink; import io.grpc.Server; @@ -28,21 +28,12 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; -/** - * 负责管理 gRPC Server 的生命周期(启动、关闭)。 - * 这个类的实现与具体的 .proto 服务定义解耦。 - */ public class PollingRpcServer { private static final Logger LOGGER = LoggerFactory.getLogger(PollingRpcServer.class); private final Server server; private final int port; - /** - * 构造函数。 - * @param serviceImpl gRPC 服务的实现实例 - * @param port 服务器监听的端口 - */ public PollingRpcServer(PixelsPollingServiceImpl serviceImpl, int port) { this.port = port; this.server = ServerBuilder.forPort(port) @@ -50,18 +41,11 @@ public PollingRpcServer(PixelsPollingServiceImpl serviceImpl, int port) { .build(); } - /** - * 启动 gRPC 服务器。 - * @throws IOException 如果端口绑定失败 - */ public void start() throws IOException { server.start(); LOGGER.info("gRPC Polling Server started, listening on port " + port); } - /** - * 平滑地关闭服务器。 - */ public void stop() { LOGGER.info("Attempting to shut down gRPC Polling Server..."); if (server != null) { @@ -82,10 +66,6 @@ public void stop() { LOGGER.info("gRPC Polling Server shut down."); } - /** - * 阻塞当前线程,直到 gRPC 服务器关闭。 - * 通常在主线程中调用,以防止应用退出。 - */ public void awaitTermination() throws InterruptedException { if (server != null) { server.awaitTermination(); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java index 1fcf6e5..d451332 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/ProtoWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.proto; @@ -50,8 +50,7 @@ * @author: AntiO2 * @date: 2025/10/5 07:10 */ -public class ProtoWriter implements PixelsSinkWriter -{ +public class ProtoWriter implements PixelsSinkWriter { private final Logger LOGGER = LoggerFactory.getLogger(ProtoWriter.class); private final RotatingWriterManager writerManager; private final TableMetadataRegistry instance; @@ -63,8 +62,7 @@ public class ProtoWriter implements PixelsSinkWriter private final Map transTracker = new ConcurrentHashMap<>(); - public ProtoWriter() throws IOException - { + public ProtoWriter() throws IOException { PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); String dataPath = sinkConfig.getSinkProtoData(); @@ -78,19 +76,16 @@ public ProtoWriter() throws IOException * * @param transId The ID of the transaction to check. */ - private void checkAndCleanupTransaction(String transId) - { + private void checkAndCleanupTransaction(String transId) { TransactionContext context = transTracker.get(transId); - if (context == null || !context.isEndReceived()) - { + if (context == null || !context.isEndReceived()) { // Transaction has not received TX END or has been cleaned up already. return; } Map tableMap = context.tableCounters; - if (tableMap == null || tableMap.isEmpty()) - { + if (tableMap == null || tableMap.isEmpty()) { // Empty transaction with no tables. Clean up immediately. transTracker.remove(transId); LOGGER.info("Transaction {} (empty) successfully completed and removed from tracker.", transId); @@ -101,17 +96,14 @@ private void checkAndCleanupTransaction(String transId) int actualProcessedRows = 0; // Iterate through all tables to check completion status - for (Map.Entry entry : tableMap.entrySet()) - { + for (Map.Entry entry : tableMap.entrySet()) { TableCounters counters = entry.getValue(); - if (!counters.isComplete()) - { + if (!counters.isComplete()) { allComplete = false; } } - if (allComplete) - { + if (allComplete) { transTracker.remove(transId); ByteBuffer transInfo = getTransBuffer(context); transInfo.rewind(); @@ -120,20 +112,16 @@ private void checkAndCleanupTransaction(String transId) } @Override - public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) - { - try - { + public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) { + try { lock.lock(); String transId = transactionMetadata.getId(); - if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) - { + if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.BEGIN) { // 1. BEGIN: Create context if not exists (in case ROWChange arrived first). TransactionContext transactionContext = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); LOGGER.debug("Transaction {} BEGIN received.", transId); transactionContext.txBegin = transactionMetadata; - } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) - { + } else if (transactionMetadata.getStatus() == SinkProto.TransactionStatus.END) { // 2. END: Finalize tracker state, merge pre-counts, and trigger cleanup. // Get existing context or create a new one (in case BEGIN was missed). @@ -141,8 +129,7 @@ public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) // --- Initialization Step: Set Total Counts --- Map newTableCounters = new ConcurrentHashMap<>(); - for (SinkProto.DataCollection dataCollection : transactionMetadata.getDataCollectionsList()) - { + for (SinkProto.DataCollection dataCollection : transactionMetadata.getDataCollectionsList()) { String fullTable = dataCollection.getDataCollection(); // Create official counter with total count newTableCounters.put(fullTable, new TableCounters((int) dataCollection.getEventCount())); @@ -152,21 +139,17 @@ public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) context.setEndReceived(newTableCounters); // --- Merge Step: Apply pre-received rows --- - for (Map.Entry preEntry : context.preEndCounts.entrySet()) - { + for (Map.Entry preEntry : context.preEndCounts.entrySet()) { String table = preEntry.getKey(); int accumulatedCount = preEntry.getValue().get(); TableCounters finalCounter = newTableCounters.get(table); - if (finalCounter != null) - { + if (finalCounter != null) { // Apply the accumulated count to the official counter - for (int i = 0; i < accumulatedCount; i++) - { + for (int i = 0; i < accumulatedCount; i++) { finalCounter.increment(); } - } else - { + } else { LOGGER.warn("Pre-received rows for table {} (count: {}) but table was not in TX END metadata. Discarding accumulated count.", table, accumulatedCount); } } @@ -177,14 +160,12 @@ public boolean writeTrans(SinkProto.TransactionMetadata transactionMetadata) checkAndCleanupTransaction(transId); } return true; - } finally - { + } finally { lock.unlock(); } } - private ByteBuffer getTransBuffer(TransactionContext transactionContext) - { + private ByteBuffer getTransBuffer(TransactionContext transactionContext) { int total = 0; byte[] transDataBegin = transactionContext.txBegin.toByteArray(); ByteBuffer beginByteBuffer = writeData(-1, transDataBegin); @@ -195,11 +176,9 @@ private ByteBuffer getTransBuffer(TransactionContext transactionContext) endByteBuffer.rewind(); total += endByteBuffer.limit(); List rowEvents = new ArrayList<>(); - for (RowChangeEvent rowChangeEvent : transactionContext.rowChangeEventList) - { + for (RowChangeEvent rowChangeEvent : transactionContext.rowChangeEventList) { ByteBuffer byteBuffer = write(rowChangeEvent.getRowRecord()); - if (byteBuffer == null) - { + if (byteBuffer == null) { return null; } byteBuffer.rewind(); @@ -208,26 +187,22 @@ private ByteBuffer getTransBuffer(TransactionContext transactionContext) } ByteBuffer buffer = ByteBuffer.allocate(total); buffer.put(beginByteBuffer.array()); - for (ByteBuffer rowEvent : rowEvents) - { + for (ByteBuffer rowEvent : rowEvents) { buffer.put(rowEvent.array()); } buffer.put(endByteBuffer.array()); return buffer; } - public ByteBuffer write(SinkProto.RowRecord rowRecord) - { + public ByteBuffer write(SinkProto.RowRecord rowRecord) { byte[] rowData = rowRecord.toByteArray(); String tableName = rowRecord.getSource().getTable(); String schemaName = rowRecord.getSource().getDb(); long tableId; - try - { + try { tableId = instance.getTableId(schemaName, tableName); - } catch (SinkException e) - { + } catch (SinkException e) { LOGGER.error("Error while getting schema table id.", e); return null; } @@ -237,23 +212,19 @@ public ByteBuffer write(SinkProto.RowRecord rowRecord) } // key: -1 means transaction, else means table id - private ByteBuffer writeData(int key, byte[] data) - { + private ByteBuffer writeData(int key, byte[] data) { ByteBuffer buf = ByteBuffer.allocate(Integer.BYTES + Integer.BYTES + data.length).order(ByteOrder.BIG_ENDIAN); // key + value len + data buf.putInt(key).putInt(data.length).put(data); return buf; } - private synchronized boolean writeBuffer(ByteBuffer buf) - { + private synchronized boolean writeBuffer(ByteBuffer buf) { PhysicalWriter writer; - try - { + try { writer = writerManager.current(); writer.prepare(buf.remaining()); writer.append(buf.array()); - } catch (IOException e) - { + } catch (IOException e) { LOGGER.error("Error while writing row record.", e); return false; } @@ -261,10 +232,8 @@ private synchronized boolean writeBuffer(ByteBuffer buf) } @Override - public boolean writeRow(RowChangeEvent rowChangeEvent) - { - try - { + public boolean writeRow(RowChangeEvent rowChangeEvent) { + try { lock.lock(); String transId = rowChangeEvent.getTransaction().getId(); String fullTable = rowChangeEvent.getFullTableName(); @@ -273,50 +242,41 @@ public boolean writeRow(RowChangeEvent rowChangeEvent) TransactionContext context = transTracker.computeIfAbsent(transId, k -> new TransactionContext()); context.rowChangeEventList.add(rowChangeEvent); // 2. Check if TX END has arrived - if (context.isEndReceived()) - { + if (context.isEndReceived()) { // TX END arrived: Use official TableCounters TableCounters counters = context.tableCounters.get(fullTable); - if (counters != null) - { + if (counters != null) { // Increment the processed row count for this table counters.increment(); // If this table completed, check if the entire transaction is complete. - if (counters.isComplete()) - { + if (counters.isComplete()) { checkAndCleanupTransaction(transId); } - } else - { + } else { LOGGER.warn("Row received for TransId {} / Table {} but was not included in TX END metadata.", transId, fullTable); } - } else - { + } else { context.incrementPreEndCount(fullTable); LOGGER.debug("Row received for TransId {} / Table {} before TX END. Accumulating count.", transId, fullTable); } return true; - } finally - { + } finally { lock.unlock(); } } @Override - public void flush() - { + public void flush() { } @Override - public void close() throws IOException - { + public void close() throws IOException { this.writerManager.close(); } - private static class TransactionContext - { + private static class TransactionContext { // Key: Full Table Name, Value: Row Count private final Map preEndCounts = new ConcurrentHashMap<>(); public List rowChangeEventList = new ArrayList<>(); @@ -327,8 +287,7 @@ private static class TransactionContext // Key: Full Table Name private Map tableCounters = null; - public void setEndReceived(Map counters) - { + public void setEndReceived(Map counters) { this.tableCounters = counters; this.endReceived = true; } @@ -336,8 +295,7 @@ public void setEndReceived(Map counters) /** * @param table Full table name */ - public void incrementPreEndCount(String table) - { + public void incrementPreEndCount(String table) { preEndCounts.computeIfAbsent(table, k -> new AtomicInteger(0)).incrementAndGet(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java index f4ecf5d..32dd2a9 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/proto/RotatingWriterManager.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.proto; @@ -36,8 +36,7 @@ * @author: AntiO2 * @date: 2025/10/5 07:34 */ -public class RotatingWriterManager -{ +public class RotatingWriterManager { private final String baseDir; private final String topic; private final int maxRecordsPerFile; @@ -47,8 +46,7 @@ public class RotatingWriterManager private PhysicalWriter currentWriter; private String currentFileName; - public RotatingWriterManager(String topic) throws IOException - { + public RotatingWriterManager(String topic) throws IOException { PixelsSinkConfig sinkConfig = PixelsSinkConfigFactory.getInstance(); this.baseDir = sinkConfig.getSinkProtoDir(); this.topic = topic; @@ -58,10 +56,8 @@ public RotatingWriterManager(String topic) throws IOException rotate(); } - private void rotate() throws IOException - { - if (currentWriter != null) - { + private void rotate() throws IOException { + if (currentWriter != null) { currentWriter.close(); registry.markFileCompleted(registry.getCurrentFileKey()); } @@ -72,20 +68,16 @@ private void rotate() throws IOException currentCount = 0; } - public PhysicalWriter current() throws IOException - { - if (currentCount >= maxRecordsPerFile) - { + public PhysicalWriter current() throws IOException { + if (currentCount >= maxRecordsPerFile) { rotate(); } currentCount++; return currentWriter; } - public void close() throws IOException - { - if (currentWriter != null) - { + public void close() throws IOException { + if (currentWriter != null) { currentWriter.close(); registry.markFileCompleted(registry.getCurrentFileKey()); } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java index 9dbb492..ab4bcfe 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaBucketDispatcher.java @@ -23,18 +23,15 @@ import io.pixelsdb.pixels.sink.event.RowChangeEvent; import io.pixelsdb.pixels.sink.writer.AbstractBucketedWriter; -public class RetinaBucketDispatcher extends AbstractBucketedWriter -{ +public class RetinaBucketDispatcher extends AbstractBucketedWriter { private final TableWriterProxy tableWriterProxy; - public RetinaBucketDispatcher() - { + public RetinaBucketDispatcher() { this.tableWriterProxy = TableWriterProxy.getInstance(); } @Override - protected void emit(RowChangeEvent event, int bucketId, SinkContext ctx) - { + protected void emit(RowChangeEvent event, int bucketId, SinkContext ctx) { tableWriterProxy .getTableWriter(event.getTable(), event.getTableId(), bucketId) .write(event, ctx); diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java index eeb694a..245e309 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaServiceProxy.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.RetinaException; @@ -38,8 +38,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; -public class RetinaServiceProxy -{ +public class RetinaServiceProxy { private static final Logger LOGGER = LoggerFactory.getLogger(RetinaServiceProxy.class); @Getter private static final PixelsSinkMode pixelsSinkMode = PixelsSinkMode.RETINA; @@ -48,49 +47,38 @@ public class RetinaServiceProxy private final AtomicBoolean isClosed = new AtomicBoolean(false); private final RetinaService retinaService; private final MetricsFacade metricsFacade = MetricsFacade.getInstance(); - private RetinaService.StreamHandler retinaStream = null; private final int vNodeId; - public RetinaServiceProxy(int bucketId) - { - if (bucketId == -1) - { + private RetinaService.StreamHandler retinaStream = null; + + public RetinaServiceProxy(int bucketId) { + if (bucketId == -1) { this.retinaService = RetinaService.Instance(); - } else - { + } else { this.retinaService = RetinaUtils.getRetinaServiceFromBucketId(bucketId); } - if (config.getRetinaWriteMode() == RetinaWriteMode.STREAM) - { + if (config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { retinaStream = retinaService.startUpdateStream(); - } else - { + } else { retinaStream = null; } this.vNodeId = BucketCache.getInstance().getRetinaNodeInfoByBucketId(bucketId).getVirtualNodeId(); } - public boolean writeTrans(String schemaName, List tableUpdateData) - { - if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) - { - try - { + public boolean writeTrans(String schemaName, List tableUpdateData) { + if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) { + try { retinaService.updateRecord(schemaName, vNodeId, tableUpdateData); - } catch (RetinaException e) - { + } catch (RetinaException e) { e.printStackTrace(); return false; } - } else - { - try - { + } else { + try { retinaStream.updateRecord(schemaName, vNodeId, tableUpdateData); - } catch (RetinaException e) - { + } catch (RetinaException e) { e.printStackTrace(); return false; } @@ -99,51 +87,38 @@ public boolean writeTrans(String schemaName, List t } public CompletableFuture writeBatchAsync - (String schemaName, List tableUpdateData) - { - if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) - { - try - { + (String schemaName, List tableUpdateData) { + if (config.getRetinaWriteMode() == RetinaWriteMode.STUB) { + try { retinaService.updateRecord(schemaName, vNodeId, tableUpdateData); - } catch (RetinaException e) - { + } catch (RetinaException e) { e.printStackTrace(); } return null; - } else - { - try - { + } else { + try { return retinaStream.updateRecord(schemaName, vNodeId, tableUpdateData); - } catch (RetinaException e) - { + } catch (RetinaException e) { e.printStackTrace(); } return null; } } - public void close() throws IOException - { + public void close() throws IOException { isClosed.compareAndSet(false, true); - if (config.getRetinaWriteMode() == RetinaWriteMode.STREAM) - { + if (config.getRetinaWriteMode() == RetinaWriteMode.STREAM) { retinaStream.close(); } } - public enum RetinaWriteMode - { + public enum RetinaWriteMode { STREAM, STUB; - public static RetinaWriteMode fromValue(String value) - { - for (RetinaWriteMode mode : values()) - { - if (mode.name().equalsIgnoreCase(value)) - { + public static RetinaWriteMode fromValue(String value) { + for (RetinaWriteMode mode : values()) { + if (mode.name().equalsIgnoreCase(value)) { return mode; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java index 96983a5..6ecc33c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/RetinaWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.sink.SinkProto; @@ -38,8 +38,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; -public class RetinaWriter implements PixelsSinkWriter -{ +public class RetinaWriter implements PixelsSinkWriter { private static final Logger LOGGER = LoggerFactory.getLogger(RetinaWriter.class); final ExecutorService dispatchExecutor = Executors.newCachedThreadPool(); private final ScheduledExecutorService timeoutScheduler = @@ -49,8 +48,7 @@ public class RetinaWriter implements PixelsSinkWriter private final SinkContextManager sinkContextManager; private final TransactionMode transactionMode; - public RetinaWriter() - { + public RetinaWriter() { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); this.sinkContextManager = SinkContextManager.getInstance(); this.flushRateLimiter = FlushRateLimiter.getInstance(); @@ -58,24 +56,18 @@ public RetinaWriter() } @Override - public boolean writeTrans(SinkProto.TransactionMetadata txMeta) - { - if(transactionMode.equals(TransactionMode.RECORD)) - { + public boolean writeTrans(SinkProto.TransactionMetadata txMeta) { + if (transactionMode.equals(TransactionMode.RECORD)) { return true; } - try - { - if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) - { + try { + if (txMeta.getStatus() == SinkProto.TransactionStatus.BEGIN) { handleTxBegin(txMeta); - } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) - { + } else if (txMeta.getStatus() == SinkProto.TransactionStatus.END) { handleTxEnd(txMeta); } - } catch (SinkException e) - { + } catch (SinkException e) { LOGGER.error(e.getMessage(), e); return false; } @@ -83,19 +75,15 @@ public boolean writeTrans(SinkProto.TransactionMetadata txMeta) } @Override - public boolean writeRow(RowChangeEvent event) - { - try - { - if (event == null) - { + public boolean writeRow(RowChangeEvent event) { + try { + if (event == null) { return false; } metricsFacade.recordRowChange(event.getTable(), event.getOp()); event.startLatencyTimer(); - if (event.getTransaction() == null || event.getTransaction().getId().isEmpty()) - { + if (event.getTransaction() == null || event.getTransaction().getId().isEmpty()) { handleNonTxEvent(event); return true; } @@ -105,21 +93,17 @@ public boolean writeRow(RowChangeEvent event) long collectionOrder = event.getTransaction().getDataCollectionOrder(); long totalOrder = event.getTransaction().getTotalOrder(); - if(transactionMode.equals(TransactionMode.RECORD)) - { + if (transactionMode.equals(TransactionMode.RECORD)) { sinkContextManager.writeRowChangeEvent(null, event); - } else - { + } else { AtomicBoolean canWrite = new AtomicBoolean(false); SinkContext ctx = sinkContextManager.getActiveTxContext(event, canWrite); - if (canWrite.get()) - { + if (canWrite.get()) { sinkContextManager.writeRowChangeEvent(ctx, event); } } - } catch (SinkException e) - { + } catch (SinkException e) { LOGGER.error(e.getMessage(), e); return false; } @@ -127,32 +111,26 @@ public boolean writeRow(RowChangeEvent event) return true; } - private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkException - { + private void handleTxBegin(SinkProto.TransactionMetadata txBegin) throws SinkException { // startTrans(txBegin.getId()).get(); - try - { + try { // flushRateLimiter.acquire(1); startTransSync(txBegin.getId()); - } catch (SinkException e) - { + } catch (SinkException e) { throw new SinkException("Failed to start trans", e); } } - private void startTransSync(String sourceTxId) throws SinkException - { + private void startTransSync(String sourceTxId) throws SinkException { sinkContextManager.startTransSync(sourceTxId); } - private void handleTxEnd(SinkProto.TransactionMetadata txEnd) - { + private void handleTxEnd(SinkProto.TransactionMetadata txEnd) { sinkContextManager.processTxCommit(txEnd); } - private void handleNonTxEvent(RowChangeEvent event) throws SinkException - { + private void handleNonTxEvent(RowChangeEvent event) throws SinkException { // virtual tx String randomId = Long.toString(System.currentTimeMillis()) + RandomUtils.nextLong(); writeTrans(buildBeginTransactionMetadata(randomId)); @@ -160,34 +138,29 @@ private void handleNonTxEvent(RowChangeEvent event) throws SinkException writeTrans(buildEndTransactionMetadata(event.getFullTableName(), randomId)); } - public void shutdown() - { + public void shutdown() { dispatchExecutor.shutdown(); timeoutScheduler.shutdown(); } @Override - public void close() throws IOException - { + public void close() throws IOException { } @Override - public void flush() - { + public void flush() { } - private SinkProto.TransactionMetadata buildBeginTransactionMetadata(String id) - { + private SinkProto.TransactionMetadata buildBeginTransactionMetadata(String id) { SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); builder.setStatus(SinkProto.TransactionStatus.BEGIN) .setId(id); return builder.build(); } - private SinkProto.TransactionMetadata buildEndTransactionMetadata(String fullTableName, String id) - { + private SinkProto.TransactionMetadata buildEndTransactionMetadata(String fullTableName, String id) { SinkProto.TransactionMetadata.Builder builder = SinkProto.TransactionMetadata.newBuilder(); builder.setStatus(SinkProto.TransactionStatus.END) .setId(id) diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java index bc3cc4d..a420efe 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContext.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.transaction.TransContext; @@ -40,8 +40,7 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -public class SinkContext -{ +public class SinkContext { private static final Logger LOGGER = LoggerFactory.getLogger(SinkContext.class); @Getter final ReentrantLock lock = new ReentrantLock(); @@ -62,63 +61,53 @@ public class SinkContext final CompletableFuture completionFuture = new CompletableFuture<>(); @Getter final TableMetadataRegistry tableMetadataRegistry = TableMetadataRegistry.Instance(); + private final Queue> recordTimes = new ConcurrentLinkedQueue<>(); @Getter Map tableCounters = new ConcurrentHashMap<>(); @Getter @Setter Queue orphanEvent = new ConcurrentLinkedQueue<>(); @Getter + @Setter + SinkProto.TransactionMetadata endTx; + @Getter private TransContext pixelsTransCtx; @Setter @Getter private boolean failed = false; - @Getter @Setter private volatile Long startTime = null; - - private final Queue> recordTimes = new ConcurrentLinkedQueue<>(); - - @Getter - @Setter - SinkProto.TransactionMetadata endTx; - public SinkContext(String sourceTxId) - { + public SinkContext(String sourceTxId) { this.sourceTxId = sourceTxId; this.pixelsTransCtx = null; setCurrStartTime(); } - public SinkContext(String sourceTxId, TransContext pixelsTransCtx) - { + public SinkContext(String sourceTxId, TransContext pixelsTransCtx) { this.sourceTxId = sourceTxId; this.pixelsTransCtx = pixelsTransCtx; setCurrStartTime(); } - void updateCounter(String table) - { + void updateCounter(String table) { updateCounter(table, 1L); } - public void setPixelsTransCtx(TransContext pixelsTransCtx) - { - if(this.pixelsTransCtx != null) - { + public void setPixelsTransCtx(TransContext pixelsTransCtx) { + if (this.pixelsTransCtx != null) { throw new IllegalStateException("Pixels Trans Context Already Set"); } this.pixelsTransCtx = pixelsTransCtx; } - public void recordTimestamp(String table, LocalDateTime timestamp) - { + public void recordTimestamp(String table, LocalDateTime timestamp) { recordTimes.offer(new Pair<>(table, timestamp)); } - public void updateCounter(String table, long count) - { + public void updateCounter(String table, long count) { tableCounterLock.lock(); tableCounters.compute(table, (k, v) -> (v == null) ? count : v + count); @@ -126,75 +115,58 @@ public void updateCounter(String table, long count) tableCounterLock.unlock(); } - public boolean isCompleted() - { - try - { + public boolean isCompleted() { + try { tableCounterLock.lock(); - if(endTx == null) - { + if (endTx == null) { return false; } - for (SinkProto.DataCollection dataCollection : endTx.getDataCollectionsList()) - { + for (SinkProto.DataCollection dataCollection : endTx.getDataCollectionsList()) { Long targetEventCount = tableCounters.get(dataCollection.getDataCollection()); long target = targetEventCount == null ? 0 : targetEventCount; LOGGER.debug("TX {}, Table {}, event count {}, tableCursors {}", endTx.getId(), dataCollection.getDataCollection(), dataCollection.getEventCount(), target); - if (dataCollection.getEventCount() > target) - { + if (dataCollection.getEventCount() > target) { return false; } } return true; - } finally - { + } finally { tableCounterLock.unlock(); } } - public int getProcessedRowsNum() - { + public int getProcessedRowsNum() { long num = 0; - try - { + try { tableCounterLock.lock(); - for(Long counter: tableCounters.values()) - { + for (Long counter : tableCounters.values()) { num += counter; } - } finally - { + } finally { tableCounterLock.unlock(); } - return (int)num; + return (int) num; } - public long getTimestamp() - { - if (pixelsTransCtx == null) - { + public long getTimestamp() { + if (pixelsTransCtx == null) { throw new RuntimeException("PixelsTransCtx is NULL"); } return pixelsTransCtx.getTimestamp(); } - public void bufferOrphanedEvent(RowChangeEvent event) - { + public void bufferOrphanedEvent(RowChangeEvent event) { orphanEvent.add(event); } - public void setCurrStartTime() - { - if (startTime != null) - { + public void setCurrStartTime() { + if (startTime != null) { return; } - synchronized (this) - { - if (startTime == null) - { + synchronized (this) { + if (startTime == null) { startTime = System.currentTimeMillis(); } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java index 6666224..13782f7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/SinkContextManager.java @@ -17,12 +17,10 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; -import io.pixelsdb.pixels.common.exception.TransException; import io.pixelsdb.pixels.common.transaction.TransContext; -import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.sink.SinkProto; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory; @@ -41,8 +39,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; -public class SinkContextManager -{ +public class SinkContextManager { private static final Logger LOGGER = LoggerFactory.getLogger(SinkContextManager.class); private static final Logger BUCKET_TRACE_LOGGER = LoggerFactory.getLogger("bucket_trace"); private static volatile SinkContextManager instance; @@ -52,28 +49,22 @@ public class SinkContextManager private final CommitMethod commitMethod; private final String freshnessLevel; private final RetinaBucketDispatcher retinaBucketDispatcher; - private SinkContextManager() - { + + private SinkContextManager() { PixelsSinkConfig config = PixelsSinkConfigFactory.getInstance(); - if (config.getCommitMethod().equals("sync")) - { + if (config.getCommitMethod().equals("sync")) { this.commitMethod = CommitMethod.Sync; - } else - { + } else { this.commitMethod = CommitMethod.Async; } this.freshnessLevel = config.getSinkMonitorFreshnessLevel(); this.retinaBucketDispatcher = new RetinaBucketDispatcher(); } - public static SinkContextManager getInstance() - { - if (instance == null) - { - synchronized (SinkContextManager.class) - { - if (instance == null) - { + public static SinkContextManager getInstance() { + if (instance == null) { + synchronized (SinkContextManager.class) { + if (instance == null) { instance = new SinkContextManager(); } } @@ -81,24 +72,19 @@ public static SinkContextManager getInstance() return instance; } - protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean canWrite) - { + protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean canWrite) { String txId = event.getTransaction().getId(); return activeTxContexts.compute(txId, (sourceTxId, sinkContext) -> { - if (sinkContext == null) - { + if (sinkContext == null) { LOGGER.trace("Allocate new tx {}\torder:{}", sourceTxId, event.getTransaction().getTotalOrder()); SinkContext newSinkContext = new SinkContext(sourceTxId); newSinkContext.bufferOrphanedEvent(event); return newSinkContext; - } else - { - try - { + } else { + try { sinkContext.getLock().lock(); - if (sinkContext.getPixelsTransCtx() == null) - { + if (sinkContext.getPixelsTransCtx() == null) { LOGGER.trace("Buffer in tx {}\torder:{}", sourceTxId, event.getTransaction().getTotalOrder()); canWrite.set(false); sinkContext.bufferOrphanedEvent(event); @@ -107,8 +93,7 @@ protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean can LOGGER.trace("Ready to write in tx {}\torder:{}", sourceTxId, event.getTransaction().getTotalOrder()); canWrite.set(true); return sinkContext; - } finally - { + } finally { sinkContext.getCond().signalAll(); sinkContext.getLock().unlock(); } @@ -117,25 +102,20 @@ protected SinkContext getActiveTxContext(RowChangeEvent event, AtomicBoolean can }); } - protected void startTransSync(String sourceTxId) - { + protected void startTransSync(String sourceTxId) { LOGGER.trace("Start trans {}", sourceTxId); TransContext pixelsTransContext = transactionProxy.getNewTransContext(sourceTxId); activeTxContexts.compute( sourceTxId, (k, oldCtx) -> { - if (oldCtx == null) - { + if (oldCtx == null) { LOGGER.trace("Start trans {} without buffered events", sourceTxId); return new SinkContext(sourceTxId, pixelsTransContext); - } else - { + } else { oldCtx.getLock().lock(); - try - { - if (oldCtx.getPixelsTransCtx() != null) - { + try { + if (oldCtx.getPixelsTransCtx() != null) { LOGGER.warn("Previous tx {} has been released, maybe due to loop process", sourceTxId); oldCtx.tableCounters = new ConcurrentHashMap<>(); } @@ -143,11 +123,9 @@ protected void startTransSync(String sourceTxId) oldCtx.setPixelsTransCtx(pixelsTransContext); handleOrphanEvents(oldCtx); oldCtx.getCond().signalAll(); - } catch (SinkException e) - { + } catch (SinkException e) { throw new RuntimeException(e); - } finally - { + } finally { oldCtx.getLock().unlock(); } return oldCtx; @@ -157,119 +135,96 @@ protected void startTransSync(String sourceTxId) LOGGER.trace("Begin Tx Sync: {}", sourceTxId); } - void processTxCommit(SinkProto.TransactionMetadata txEnd) - { + void processTxCommit(SinkProto.TransactionMetadata txEnd) { String txId = txEnd.getId(); SinkContext ctx = getSinkContext(txId); - if (ctx == null) - { + if (ctx == null) { throw new RuntimeException("Sink Context is null"); } - try - { + try { ctx.tableCounterLock.lock(); ctx.setEndTx(txEnd); long startTs = System.currentTimeMillis(); - if(ctx.isCompleted()) - { + if (ctx.isCompleted()) { endTransaction(ctx); } - } finally - { + } finally { ctx.tableCounterLock.unlock(); } } - void endTransaction(SinkContext ctx) - { + void endTransaction(SinkContext ctx) { String txId = ctx.getSourceTxId(); removeSinkContext(txId); boolean failed = ctx.isFailed(); - if (!failed) - { + if (!failed) { LOGGER.trace("Committed transaction: {}, Pixels Trans is {}", txId, ctx.getPixelsTransCtx().getTransId()); - switch (commitMethod) - { - case Sync -> - { + switch (commitMethod) { + case Sync -> { transactionProxy.commitTransSync(ctx); } - case Async -> - { + case Async -> { transactionProxy.commitTransAsync(ctx); } } - if(freshnessLevel.equals("embed")) - { - for(String table: ctx.getTableCounters().keySet()) - { + if (freshnessLevel.equals("embed")) { + for (String table : ctx.getTableCounters().keySet()) { String tableName = DataTransform.extractTableName(table); FreshnessClient.getInstance().addMonitoredTable(tableName); } } - } else - { + } else { LOGGER.info("Abort transaction: {}", txId); CompletableFuture.runAsync(() -> { transactionProxy.rollbackTrans(ctx.getPixelsTransCtx()); }).whenComplete((v, ex) -> { - if (ex != null) - { + if (ex != null) { LOGGER.error("Rollback failed", ex); } }); } } - private void handleOrphanEvents(SinkContext ctx) throws SinkException - { + private void handleOrphanEvents(SinkContext ctx) throws SinkException { Queue buffered = ctx.getOrphanEvent(); ctx.setOrphanEvent(null); - if (buffered != null) - { + if (buffered != null) { LOGGER.trace("Handle Orphan Events in {}", ctx.sourceTxId); - for (RowChangeEvent event : buffered) - { + for (RowChangeEvent event : buffered) { writeRowChangeEvent(ctx, event); } } } - protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException - { - if(ctx != null) - { + protected void writeRowChangeEvent(SinkContext ctx, RowChangeEvent event) throws SinkException { + if (ctx != null) { event.setTimeStamp(ctx.getTimestamp()); } retinaBucketDispatcher.writeRowChangeEvent(event, ctx); } - protected SinkContext getSinkContext(String txId) - { + protected SinkContext getSinkContext(String txId) { return activeTxContexts.get(txId); } - protected void removeSinkContext(String txId) - { + protected void removeSinkContext(String txId) { activeTxContexts.remove(txId); } - protected void writeRandomRowChangeEvent(String randomId, RowChangeEvent event) throws SinkException - { + protected void writeRandomRowChangeEvent(String randomId, RowChangeEvent event) throws SinkException { writeRowChangeEvent(getSinkContext(randomId), event); } - public int getActiveTxnsNum() - { + public int getActiveTxnsNum() { return activeTxContexts.size(); } - public String findMinActiveTx() - { - Comparator customComparator = (key1, key2) -> { + public String findMinActiveTx() { + Comparator customComparator = (key1, key2) -> + { try { String[] parts1 = key1.split("_"); int int1 = Integer.parseInt(parts1[0]); @@ -294,8 +249,7 @@ public String findMinActiveTx() return min.orElse("None"); } - private enum CommitMethod - { + private enum CommitMethod { Sync, Async } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java index 8d35cd7..51c597c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableCrossTxWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; @@ -41,15 +41,13 @@ * @author: AntiO2 * @date: 2025/9/27 09:36 */ -public class TableCrossTxWriter extends TableWriter -{ +public class TableCrossTxWriter extends TableWriter { + protected final ReentrantLock writeLock = new ReentrantLock(); @Getter private final Logger LOGGER = LoggerFactory.getLogger(TableCrossTxWriter.class); private final int flushBatchSize; - protected final ReentrantLock writeLock = new ReentrantLock(); - public TableCrossTxWriter(String t, int bucketId) - { + public TableCrossTxWriter(String t, int bucketId) { super(t, bucketId); flushBatchSize = config.getFlushBatchSize(); } @@ -57,27 +55,21 @@ public TableCrossTxWriter(String t, int bucketId) /** * Flush any buffered events for the current transaction. */ - public void flush(List batch) - { + public void flush(List batch) { writeLock.lock(); - try - { + try { String txId = null; List smallBatch = null; List txIds = new ArrayList<>(); List fullTableName = new ArrayList<>(); List tableUpdateDataBuilderList = new LinkedList<>(); List tableUpdateCount = new ArrayList<>(); - for (RowChangeEvent event : batch) - { + for (RowChangeEvent event : batch) { String currTxId = event.getTransaction().getId(); - if (!currTxId.equals(txId)) - { - if (smallBatch != null && !smallBatch.isEmpty()) - { + if (!currTxId.equals(txId)) { + if (smallBatch != null && !smallBatch.isEmpty()) { RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(txId, smallBatch); - if (builder == null) - { + if (builder == null) { continue; } tableUpdateDataBuilderList.add(builder); @@ -91,11 +83,9 @@ public void flush(List batch) smallBatch.add(event); } - if (smallBatch != null) - { + if (smallBatch != null) { RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(txId, smallBatch); - if (builder != null) - { + if (builder != null) { tableUpdateDataBuilderList.add(buildTableUpdateDataFromBatch(txId, smallBatch)); tableUpdateCount.add(smallBatch.size()); } @@ -112,8 +102,7 @@ public void flush(List batch) // } List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); - for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) - { + for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) { tableUpdateData.add(tableUpdateDataItem.build()); } CompletableFuture updateRecordResponseCompletableFuture = @@ -122,84 +111,66 @@ public void flush(List batch) updateRecordResponseCompletableFuture.thenAccept( resp -> { - if (resp.getHeader().getErrorCode() != 0) - { + if (resp.getHeader().getErrorCode() != 0) { failCtxs(txIds); - } else - { + } else { long txEndTime = System.currentTimeMillis(); - if (freshnessLevel.equals("row")) - { + if (freshnessLevel.equals("row")) { metricsFacade.recordFreshness(txEndTime - txStartTime); } updateCtxCounters(txIds, fullTableName, tableUpdateCount); } } ); - } finally - { + } finally { writeLock.unlock(); } } - private void failCtxs(List txIds) - { - for (String writeTxId : txIds) - { + private void failCtxs(List txIds) { + for (String writeTxId : txIds) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); - if (sinkContext != null) - { + if (sinkContext != null) { sinkContext.setFailed(true); } } } - private void updateCtxCounters(List txIds, List fullTableName, List tableUpdateCount) - { + private void updateCtxCounters(List txIds, List fullTableName, List tableUpdateCount) { writeLock.lock(); - for (int i = 0; i < txIds.size(); i++) - { + for (int i = 0; i < txIds.size(); i++) { metricsFacade.recordRowEvent(tableUpdateCount.get(i)); String writeTxId = txIds.get(i); SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(writeTxId); - try - { + try { sinkContext.tableCounterLock.lock(); sinkContext.recordTimestamp(fullTableName.get(i), LocalDateTime.now()); sinkContext.updateCounter(fullTableName.get(i), tableUpdateCount.get(i)); - if(sinkContext.isCompleted()) - { + if (sinkContext.isCompleted()) { SinkContextManager.getInstance().endTransaction(sinkContext); } - } finally - { + } finally { sinkContext.tableCounterLock.unlock(); } } writeLock.unlock(); } - protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) - { + protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(String txId, List smallBatch) { SinkContext sinkContext = SinkContextManager.getInstance().getSinkContext(txId); - if (sinkContext == null) - { + if (sinkContext == null) { return null; } - try - { + try { sinkContext.getLock().lock(); - while (sinkContext.getPixelsTransCtx() == null) - { + while (sinkContext.getPixelsTransCtx() == null) { LOGGER.warn("Wait for tx to begin trans: {}", txId); // CODE SHOULD NOT REACH HERE sinkContext.getCond().await(); } - } catch (InterruptedException e) - { + } catch (InterruptedException e) { throw new RuntimeException(e); - } finally - { + } finally { sinkContext.getLock().unlock(); } RowChangeEvent event1 = smallBatch.get(0); @@ -208,22 +179,18 @@ protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(Stri .setTimestamp(sinkContext.getTimestamp()) .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) .setTableName(tableName); - try - { - for (RowChangeEvent smallEvent : smallBatch) - { + try { + for (RowChangeEvent smallEvent : smallBatch) { addUpdateData(smallEvent, builder); } - } catch (SinkException e) - { + } catch (SinkException e) { throw new RuntimeException("Flush failed for table " + tableName, e); } return builder; } @Override - protected boolean needFlush() - { + protected boolean needFlush() { return buffer.size() >= flushBatchSize; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java index ffddec9..773181b 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleRecordWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.transaction.TransContext; @@ -35,14 +35,12 @@ import java.util.List; import java.util.concurrent.CompletableFuture; -public class TableSingleRecordWriter extends TableCrossTxWriter -{ +public class TableSingleRecordWriter extends TableCrossTxWriter { @Getter private final Logger LOGGER = LoggerFactory.getLogger(TableSingleRecordWriter.class); private final TransactionProxy transactionProxy; - public TableSingleRecordWriter(String t, int bucketId) - { + public TableSingleRecordWriter(String t, int bucketId) { super(t, bucketId); this.transactionProxy = TransactionProxy.Instance(); } @@ -50,22 +48,18 @@ public TableSingleRecordWriter(String t, int bucketId) /** * Flush any buffered events for the current transaction. */ - public void flush(List batch) - { + public void flush(List batch) { TransContext pixelsTransContext = transactionProxy.getNewTransContext(tableName); writeLock.lock(); - try - { + try { List tableUpdateDataBuilderList = new LinkedList<>(); - for (RowChangeEvent event : batch) - { + for (RowChangeEvent event : batch) { event.setTimeStamp(pixelsTransContext.getTimestamp()); event.updateIndexKey(); } RetinaProto.TableUpdateData.Builder builder = buildTableUpdateDataFromBatch(pixelsTransContext, batch); - if (builder != null) - { + if (builder != null) { tableUpdateDataBuilderList.add(builder); } @@ -73,8 +67,7 @@ public void flush(List batch) long txStartTime = System.currentTimeMillis(); List tableUpdateData = new ArrayList<>(tableUpdateDataBuilderList.size()); - for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) - { + for (RetinaProto.TableUpdateData.Builder tableUpdateDataItem : tableUpdateDataBuilderList) { tableUpdateData.add(tableUpdateDataItem.build()); } @@ -84,54 +77,43 @@ public void flush(List batch) updateRecordResponseCompletableFuture.thenAccept( resp -> { - if(freshness_embed) - { + if (freshness_embed) { FreshnessClient.getInstance().addMonitoredTable(tableName); } - if (resp.getHeader().getErrorCode() != 0) - { + if (resp.getHeader().getErrorCode() != 0) { transactionProxy.rollbackTrans(pixelsTransContext); - } else - { + } else { metricsFacade.recordRowEvent(batch.size()); long txEndTime = System.currentTimeMillis(); - if (freshnessLevel.equals("row")) - { + if (freshnessLevel.equals("row")) { metricsFacade.recordFreshness(txEndTime - txStartTime); } transactionProxy.commitTrans(pixelsTransContext); - if(startWriteLatencyTimer != null) - { + if (startWriteLatencyTimer != null) { startWriteLatencyTimer.observeDuration(); } } } ); - } catch (SinkException e) - { + } catch (SinkException e) { throw new RuntimeException(e); - } finally - { + } finally { writeLock.unlock(); } } - protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(TransContext transContext, List smallBatch) - { + protected RetinaProto.TableUpdateData.Builder buildTableUpdateDataFromBatch(TransContext transContext, List smallBatch) { RowChangeEvent event1 = smallBatch.get(0); RetinaProto.TableUpdateData.Builder builder = RetinaProto.TableUpdateData.newBuilder() .setTimestamp(transContext.getTimestamp()) .setPrimaryIndexId(event1.getTableMetadata().getPrimaryIndexKeyId()) .setTableName(tableName); - try - { - for (RowChangeEvent smallEvent : smallBatch) - { + try { + for (RowChangeEvent smallEvent : smallBatch) { addUpdateData(smallEvent, builder); } - } catch (SinkException e) - { + } catch (SinkException e) { throw new RuntimeException("Flush failed for table " + tableName, e); } return builder; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java index 8afd8ce..ba61d69 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableSingleTxWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.retina.RetinaProto; @@ -30,31 +30,26 @@ import java.util.ArrayList; import java.util.List; -public class TableSingleTxWriter extends TableWriter -{ +public class TableSingleTxWriter extends TableWriter { private static final long TX_TIMEOUT_MS = 3000; @Getter private final Logger LOGGER = LoggerFactory.getLogger(TableSingleTxWriter.class); - public TableSingleTxWriter(String tableName, int bucketId) - { + public TableSingleTxWriter(String tableName, int bucketId) { super(tableName, bucketId); } /** * Flush any buffered events for the current transaction. */ - public void flush(List batchToFlush) - { + public void flush(List batchToFlush) { List batch; String txId; RetinaProto.TableUpdateData.Builder toBuild; SinkContext sinkContext = null; bufferLock.lock(); - try - { - if (buffer.isEmpty() || currentTxId == null) - { + try { + if (buffer.isEmpty() || currentTxId == null) { return; } txId = currentTxId; @@ -62,26 +57,21 @@ public void flush(List batchToFlush) sinkContext = SinkContextManager.getInstance().getSinkContext(txId); sinkContext.getLock().lock(); - try - { - while (sinkContext.getPixelsTransCtx() == null) - { + try { + while (sinkContext.getPixelsTransCtx() == null) { LOGGER.warn("Wait for prev tx to begin trans: {}", txId); sinkContext.getCond().await(); } - } finally - { + } finally { sinkContext.getLock().unlock(); } // Swap buffers quickly under lock batch = buffer; buffer = new ArrayList<>(); - } catch (InterruptedException e) - { + } catch (InterruptedException e) { throw new RuntimeException(e); - } finally - { + } finally { bufferLock.unlock(); } @@ -92,10 +82,8 @@ public void flush(List batchToFlush) .setTableName(tableName); - try - { - for (RowChangeEvent event : batch) - { + try { + for (RowChangeEvent event : batch) { addUpdateData(event, builder); } List tableUpdateData = List.of(builder.build()); @@ -103,17 +91,14 @@ public void flush(List batchToFlush) sinkContext.updateCounter(fullTableName, batch.size()); // ---- Outside lock: build proto and write ---- LOGGER.info("Flushing {} events for table {} txId={}", batch.size(), fullTableName, txId); - } catch (SinkException e) - { + } catch (SinkException e) { throw new RuntimeException("Flush failed for table " + tableName, e); } } @Override - protected boolean needFlush() - { - if (currentTxId == null || !currentTxId.equals(txId)) - { + protected boolean needFlush() { + if (currentTxId == null || !currentTxId.equals(txId)) { return !buffer.isEmpty(); } return false; diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java index 01ee192..cbd251d 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriter.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; @@ -46,22 +46,22 @@ * @author: AntiO2 * @date: 2025/9/27 09:58 */ -public abstract class TableWriter -{ +public abstract class TableWriter { protected final RetinaServiceProxy delegate; // physical writer - - private final ScheduledExecutorService flushExecutor = Executors.newSingleThreadScheduledExecutor(); - private final ScheduledExecutorService logScheduler = Executors.newScheduledThreadPool(1); protected final ReentrantLock bufferLock = new ReentrantLock(); protected final Condition flushCondition = bufferLock.newCondition(); protected final Thread flusherThread; - protected volatile boolean running = true; protected final String tableName; protected final long flushInterval; protected final FlushRateLimiter flushRateLimiter; protected final SinkContextManager sinkContextManager; protected final String freshnessLevel; + protected final boolean freshness_embed; + private final ScheduledExecutorService flushExecutor = Executors.newSingleThreadScheduledExecutor(); + private final ScheduledExecutorService logScheduler = Executors.newScheduledThreadPool(1); + private final AtomicInteger counter = new AtomicInteger(); + protected volatile boolean running = true; // Shared state (protected by lock) protected List buffer = new LinkedList<>(); protected volatile String currentTxId = null; @@ -70,10 +70,8 @@ public abstract class TableWriter protected PixelsSinkConfig config; protected MetricsFacade metricsFacade = MetricsFacade.getInstance(); protected TransactionMode transactionMode; - private final AtomicInteger counter = new AtomicInteger(); - protected final boolean freshness_embed; - protected TableWriter(String tableName, int bucketId) - { + + protected TableWriter(String tableName, int bucketId) { this.config = PixelsSinkConfigFactory.getInstance(); this.tableName = tableName; this.flushInterval = config.getFlushIntervalMs(); @@ -83,15 +81,12 @@ protected TableWriter(String tableName, int bucketId) this.delegate = new RetinaServiceProxy(bucketId); this.transactionMode = config.getTransactionMode(); String sinkMonitorFreshnessLevel = config.getSinkMonitorFreshnessLevel(); - if(sinkMonitorFreshnessLevel.equals("embed")) - { + if (sinkMonitorFreshnessLevel.equals("embed")) { freshness_embed = true; - } else - { + } else { freshness_embed = false; } - if(this.config.isMonitorReportEnabled() && this.config.isRetinaLogQueueEnabled()) - { + if (this.config.isMonitorReportEnabled() && this.config.isRetinaLogQueueEnabled()) { long interval = this.config.getMonitorReportInterval(); Runnable monitorTask = writerInfoTask(tableName); logScheduler.scheduleAtFixedRate( @@ -105,53 +100,50 @@ protected TableWriter(String tableName, int bucketId) this.flusherThread.start(); } - private class FlusherRunnable implements Runnable { - @Override - public void run() { - while (running) { - bufferLock.lock(); - try { - if (!needFlush()) - { - try - { - // Conditional wait: will wait until signaled by write() or timeout - flushCondition.await(flushInterval, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - // Exit loop if interrupted during shutdown - running = false; - Thread.currentThread().interrupt(); - return; - } - } - - List batchToFlush = buffer; - buffer = new LinkedList<>(); - bufferLock.unlock(); - submitFlushTask(batchToFlush); - bufferLock.lock(); - } finally - { - bufferLock.unlock(); - } + /** + * Helper: add insert/delete data into proto builder. + */ + protected static void addUpdateData(RowChangeEvent rowChangeEvent, + RetinaProto.TableUpdateData.Builder builder) throws SinkException { + switch (rowChangeEvent.getOp()) { + case SNAPSHOT, INSERT -> { + RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); + insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); + insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); + builder.addInsertData(insertDataBuilder); + } + case UPDATE -> { + RetinaProto.UpdateData.Builder updateDataBuilder = RetinaProto.UpdateData.newBuilder(); + updateDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); + updateDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); + builder.addUpdateData(updateDataBuilder); + } + case DELETE -> { + RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); + deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); + builder.addDeleteData(deleteDataBuilder); + } + case UNRECOGNIZED -> { + throw new SinkException("Unrecognized op: " + rowChangeEvent.getOp()); } } } - private void submitFlushTask(List batch) - { - if(batch == null || batch.isEmpty()) - { + + private void submitFlushTask(List batch) { + if (batch == null || batch.isEmpty()) { return; } - flushExecutor.submit(() -> { - flush(batch); + flushExecutor.submit(() -> + { + flush(batch); }); } - private Runnable writerInfoTask(String tableName) - { + + private Runnable writerInfoTask(String tableName) { final AtomicInteger reportId = new AtomicInteger(); final AtomicInteger lastRunCounter = new AtomicInteger(); - Runnable monitorTask = () -> { + Runnable monitorTask = () -> + { String firstTx = "none"; RowChangeEvent firstEvent = null; int len = 0; @@ -161,8 +153,7 @@ private Runnable writerInfoTask(String tableName) firstEvent = buffer.get(0); } bufferLock.unlock(); - if(firstEvent != null) - { + if (firstEvent != null) { firstTx = firstEvent.getTransaction().getId(); int count = counter.get(); getLOGGER().info("{} Writer {}: Tx Now is {}. Buffer Len is {}. Total Count {}", reportId.incrementAndGet(), tableName, firstTx, len, count); @@ -171,73 +162,30 @@ private Runnable writerInfoTask(String tableName) return monitorTask; } - /** - * Helper: add insert/delete data into proto builder. - */ - protected static void addUpdateData(RowChangeEvent rowChangeEvent, - RetinaProto.TableUpdateData.Builder builder) throws SinkException - { - switch (rowChangeEvent.getOp()) - { - case SNAPSHOT, INSERT -> - { - RetinaProto.InsertData.Builder insertDataBuilder = RetinaProto.InsertData.newBuilder(); - insertDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); - insertDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); - builder.addInsertData(insertDataBuilder); - } - case UPDATE -> - { - RetinaProto.UpdateData.Builder updateDataBuilder = RetinaProto.UpdateData.newBuilder(); - updateDataBuilder.addIndexKeys(rowChangeEvent.getAfterKey()); - updateDataBuilder.addAllColValues(rowChangeEvent.getAfterData()); - builder.addUpdateData(updateDataBuilder); - } - case DELETE -> - { - RetinaProto.DeleteData.Builder deleteDataBuilder = RetinaProto.DeleteData.newBuilder(); - deleteDataBuilder.addIndexKeys(rowChangeEvent.getBeforeKey()); - builder.addDeleteData(deleteDataBuilder); - } - case UNRECOGNIZED -> - { - throw new SinkException("Unrecognized op: " + rowChangeEvent.getOp()); - } - } - } - protected abstract Logger getLOGGER(); - public boolean write(RowChangeEvent event, SinkContext ctx) - { - try - { + public boolean write(RowChangeEvent event, SinkContext ctx) { + try { bufferLock.lock(); - try - { - if(!transactionMode.equals(TransactionMode.RECORD)) - { + try { + if (!transactionMode.equals(TransactionMode.RECORD)) { txId = ctx.getSourceTxId(); } currentTxId = txId; - if (fullTableName == null) - { + if (fullTableName == null) { fullTableName = event.getFullTableName(); } counter.incrementAndGet(); buffer.add(event); - if (needFlush()) - { + if (needFlush()) { flushCondition.signalAll(); } - } finally - { + } finally { bufferLock.unlock(); } return true; - } catch (Exception e) - { + } catch (Exception e) { getLOGGER().error("Write failed for table {}", tableName, e); return false; } @@ -247,28 +195,52 @@ public boolean write(RowChangeEvent event, SinkContext ctx) protected abstract boolean needFlush(); - public void close() - { + public void close() { this.running = false; - if (this.flusherThread != null) - { + if (this.flusherThread != null) { this.flusherThread.interrupt(); } logScheduler.shutdown(); - try - { + try { logScheduler.awaitTermination(5, TimeUnit.SECONDS); flushExecutor.awaitTermination(5, TimeUnit.SECONDS); - if (this.flusherThread != null) - { + if (this.flusherThread != null) { this.flusherThread.join(5000); } delegate.close(); - } catch (InterruptedException ignored) - { - } catch (IOException e) - { + } catch (InterruptedException ignored) { + } catch (IOException e) { throw new RuntimeException(e); } } + + private class FlusherRunnable implements Runnable { + @Override + public void run() { + while (running) { + bufferLock.lock(); + try { + if (!needFlush()) { + try { + // Conditional wait: will wait until signaled by write() or timeout + flushCondition.await(flushInterval, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // Exit loop if interrupted during shutdown + running = false; + Thread.currentThread().interrupt(); + return; + } + } + + List batchToFlush = buffer; + buffer = new LinkedList<>(); + bufferLock.unlock(); + submitFlushTask(batchToFlush); + bufferLock.lock(); + } finally { + bufferLock.unlock(); + } + } + } + } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java index 0fe7f17..3733e3c 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxy.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.node.BucketCache; @@ -28,57 +28,47 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -public class TableWriterProxy -{ +public class TableWriterProxy { private final static TableWriterProxy INSTANCE = new TableWriterProxy(); private final TransactionMode transactionMode; private final int retinaCliNum; private final Map WRITER_REGISTRY = new ConcurrentHashMap<>(); - private TableWriterProxy() - { + private TableWriterProxy() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); this.transactionMode = pixelsSinkConfig.getTransactionMode(); this.retinaCliNum = pixelsSinkConfig.getRetinaClientNum(); } - protected static TableWriterProxy getInstance() - { + protected static TableWriterProxy getInstance() { return INSTANCE; } - protected TableWriter getTableWriter(String tableName, long tableId, int bucket) - { + protected TableWriter getTableWriter(String tableName, long tableId, int bucket) { int cliNo = bucket % retinaCliNum; // warn: we assume table id is less than INT.MAX WriterKey key = new WriterKey(tableId, BucketCache.getInstance().getRetinaNodeInfoByBucketId(bucket), cliNo); return WRITER_REGISTRY.computeIfAbsent(key, t -> { - switch (transactionMode) - { - case SINGLE -> - { + switch (transactionMode) { + case SINGLE -> { return new TableSingleTxWriter(tableName, bucket); } - case BATCH -> - { + case BATCH -> { return new TableCrossTxWriter(tableName, bucket); } - case RECORD -> - { + case RECORD -> { return new TableSingleRecordWriter(tableName, bucket); } - default -> - { + default -> { throw new IllegalArgumentException("Unknown transaction mode: " + transactionMode); } } }); } - record WriterKey(long tableId, NodeProto.NodeInfo nodeInfo, int cliNo) - { + record WriterKey(long tableId, NodeProto.NodeInfo nodeInfo, int cliNo) { } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java index 8a873a3..145095e 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionMode.java @@ -17,21 +17,17 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; -public enum TransactionMode -{ +public enum TransactionMode { SINGLE, RECORD, BATCH; - public static TransactionMode fromValue(String value) - { - for (TransactionMode mode : values()) - { - if (mode.name().equalsIgnoreCase(value)) - { + public static TransactionMode fromValue(String value) { + for (TransactionMode mode : values()) { + if (mode.name().equalsIgnoreCase(value)) { return mode; } } diff --git a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java index 1a94514..d1092c7 100644 --- a/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java +++ b/src/main/java/io/pixelsdb/pixels/sink/writer/retina/TransactionProxy.java @@ -17,7 +17,7 @@ * License along with Pixels. If not, see * . */ - + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.common.exception.TransException; @@ -40,8 +40,7 @@ * * @author AntiO2 */ -public class TransactionProxy -{ +public class TransactionProxy { private static final Logger LOGGER = LoggerFactory.getLogger(TransactionProxy.class); private static volatile TransactionProxy instance; private final TransService transService; @@ -58,8 +57,7 @@ public class TransactionProxy private AtomicInteger beginCount = new AtomicInteger(0); private AtomicInteger commitCount = new AtomicInteger(0); - private TransactionProxy() - { + private TransactionProxy() { PixelsSinkConfig pixelsSinkConfig = PixelsSinkConfigFactory.getInstance(); BATCH_SIZE = pixelsSinkConfig.getCommitBatchSize(); WORKER_COUNT = pixelsSinkConfig.getCommitBatchWorkers(); @@ -78,22 +76,17 @@ private TransactionProxy() return t; } ); - for (int i = 0; i < WORKER_COUNT; i++) - { + for (int i = 0; i < WORKER_COUNT; i++) { batchCommitExecutor.submit(this::batchCommitWorker); } this.freshnessLevel = pixelsSinkConfig.getSinkMonitorFreshnessLevel(); } - public static TransactionProxy Instance() - { - if (instance == null) - { - synchronized (TransactionProxy.class) - { - if (instance == null) - { + public static TransactionProxy Instance() { + if (instance == null) { + synchronized (TransactionProxy.class) { + if (instance == null) { instance = new TransactionProxy(); } } @@ -101,62 +94,48 @@ public static TransactionProxy Instance() return instance; } - public static void staticClose() - { - if (instance != null) - { + public static void staticClose() { + if (instance != null) { instance.close(); } } - private void requestTransactions() - { - try - { + private void requestTransactions() { + try { List newContexts = transService.beginTransBatch(1000, false); transContextQueue.addAll(newContexts); - } catch (TransException e) - { + } catch (TransException e) { throw new RuntimeException("Batch request failed", e); } } @Deprecated - public TransContext getNewTransContext() - { + public TransContext getNewTransContext() { return getNewTransContext("None"); } - public TransContext getNewTransContext(String txId) - { + public TransContext getNewTransContext(String txId) { beginCount.incrementAndGet(); - if(true) - { - try - { + if (true) { + try { TransContext transContext = transService.beginTrans(false); LOGGER.trace("{} begin {}", txId, transContext.getTransId()); return transContext; - } catch (TransException e) - { + } catch (TransException e) { throw null; } } TransContext ctx = transContextQueue.poll(); - if (ctx != null) - { + if (ctx != null) { return ctx; } - synchronized (batchLock) - { + synchronized (batchLock) { ctx = transContextQueue.poll(); - if (ctx == null) - { + if (ctx == null) { requestTransactions(); ctx = transContextQueue.poll(); - if (ctx == null) - { + if (ctx == null) { throw new IllegalStateException("No contexts available"); } } @@ -164,55 +143,43 @@ public TransContext getNewTransContext(String txId) } } - public void commitTransAsync(SinkContext transContext) - { + public void commitTransAsync(SinkContext transContext) { toCommitTransContextQueue.add(transContext); } - public void commitTransSync(SinkContext transContext) - { + public void commitTransSync(SinkContext transContext) { commitTrans(transContext.getPixelsTransCtx()); metricsFacade.recordTransaction(); long txEndTime = System.currentTimeMillis(); - if (freshnessLevel.equals("txn")) - { + if (freshnessLevel.equals("txn")) { metricsFacade.recordFreshness(txEndTime - transContext.getStartTime()); } } - public void commitTrans(TransContext ctx) - { + public void commitTrans(TransContext ctx) { commitCount.incrementAndGet(); - try - { + try { transService.commitTrans(ctx.getTransId(), false); - } catch (TransException e) - { + } catch (TransException e) { LOGGER.error("Batch commit failed: {}", e.getMessage(), e); } } - public void rollbackTrans(TransContext ctx) - { - try - { + public void rollbackTrans(TransContext ctx) { + try { transService.rollbackTrans(ctx.getTransId(), false); - } catch (TransException e) - { + } catch (TransException e) { LOGGER.error("Rollback transaction failed: {}", e.getMessage(), e); } } - private void batchCommitWorker() - { + private void batchCommitWorker() { List batchTransIds = new ArrayList<>(BATCH_SIZE); List batchContexts = new ArrayList<>(BATCH_SIZE); List txStartTimes = new ArrayList<>(BATCH_SIZE); - while (true) - { - try - { + while (true) { + try { batchContexts.clear(); batchTransIds.clear(); txStartTimes.clear(); @@ -224,18 +191,15 @@ private void batchCommitWorker() txStartTimes.add(firstSinkContext.getStartTime()); long startTime = System.nanoTime(); - while (batchContexts.size() < BATCH_SIZE) - { + while (batchContexts.size() < BATCH_SIZE) { long elapsedMs = (System.nanoTime() - startTime) / 1_000_000; long remainingMs = MAX_WAIT_MS - elapsedMs; - if (remainingMs <= 0) - { + if (remainingMs <= 0) { break; } SinkContext ctx = toCommitTransContextQueue.poll(remainingMs, TimeUnit.MILLISECONDS); - if (ctx == null) - { + if (ctx == null) { break; } transContext = ctx.getPixelsTransCtx(); @@ -248,8 +212,7 @@ private void batchCommitWorker() metricsFacade.recordTransaction(batchTransIds.size()); long txEndTime = System.currentTimeMillis(); - if (freshnessLevel.equals("txn")) - { + if (freshnessLevel.equals("txn")) { txStartTimes.forEach( txStartTime -> { @@ -257,44 +220,34 @@ private void batchCommitWorker() } ); } - if (LOGGER.isTraceEnabled()) - { + if (LOGGER.isTraceEnabled()) { LOGGER.trace("[{}] Batch committed {} transactions ({} waited ms)", Thread.currentThread().getName(), batchTransIds.size(), (System.nanoTime() - startTime) / 1_000_000); } - } catch (InterruptedException ie) - { + } catch (InterruptedException ie) { LOGGER.warn("Batch commit worker interrupted, exiting..."); Thread.currentThread().interrupt(); break; - } catch (TransException e) - { + } catch (TransException e) { LOGGER.error("Batch commit failed: {}", e.getMessage(), e); - } catch (Exception e) - { + } catch (Exception e) { LOGGER.error("Unexpected error in batch commit worker", e); } } } - public void close() - { - synchronized (batchLock) - { - while (true) - { + public void close() { + synchronized (batchLock) { + while (true) { TransContext ctx = transContextQueue.poll(); - if (ctx == null) - { + if (ctx == null) { break; } - try - { + try { transService.rollbackTrans(ctx.getTransId(), false); - } catch (TransException e) - { + } catch (TransException e) { throw new RuntimeException(e); } } From b7bca9cff5880d701ab3bcffc786115af0da871c Mon Sep 17 00:00:00 2001 From: AntiO2 Date: Wed, 24 Dec 2025 13:45:08 +0000 Subject: [PATCH 53/53] Test class add license header --- .../concurrent/TransactionServiceTest.java | 20 +++++++++++++++++++ .../pixels/sink/event/RowChangeEventTest.java | 20 +++++++++++++++++++ .../sink/event/deserializer/RowBatchTest.java | 20 +++++++++++++++++++ .../sink/freshness/TestFreshnessClient.java | 20 +++++++++++++++++++ .../pixels/sink/writer/TestRetinaWriter.java | 20 +++++++++++++++++++ .../pixelsdb/pixels/sink/writer/TpcHTest.java | 20 +++++++++++++++++++ .../writer/retina/TableWriterProxyTest.java | 20 +++++++++++++++++++ 7 files changed, 140 insertions(+) diff --git a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java index d725a03..0f590f1 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/concurrent/TransactionServiceTest.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.concurrent; diff --git a/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java index 4ce0766..5ae043d 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/RowChangeEventTest.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.event; import com.google.protobuf.ByteString; diff --git a/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java index ca20e54..0c36b72 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/event/deserializer/RowBatchTest.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.event.deserializer; import io.pixelsdb.pixels.core.TypeDescription; diff --git a/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java b/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java index 6535ad8..da55de3 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java +++ b/src/test/java/io/pixelsdb/pixels/sink/freshness/TestFreshnessClient.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.freshness; import io.pixelsdb.pixels.sink.config.PixelsSinkConfig; diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java index db64702..15e43ea 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TestRetinaWriter.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.writer; import com.google.protobuf.ByteString; diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java b/src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java index c43d220..a3edbca 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/TpcHTest.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.writer; import io.pixelsdb.pixels.common.exception.RetinaException; diff --git a/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java b/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java index af74333..169c94b 100644 --- a/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java +++ b/src/test/java/io/pixelsdb/pixels/sink/writer/retina/TableWriterProxyTest.java @@ -1,3 +1,23 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + package io.pixelsdb.pixels.sink.writer.retina; import io.pixelsdb.pixels.sink.config.factory.PixelsSinkConfigFactory;