You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2022/08/05 07:33:08 UTC

[GitHub] [inlong] iamsee123 opened a new pull request, #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

iamsee123 opened a new pull request, #5367:
URL: https://github.com/apache/inlong/pull/5367

   ### Prepare a Pull Request
   - Fixes #5046 
   
   ### Motivation
   
   Add the collecting postgreSQL data function to agent module and provide the necessary management ability for manager module.
   
   ### Modifications
   
   - PostgreSqlReader: read data from postgres by Debezium
   - PostgreSqlSource: PostgreSql source, split PostgreSql source job into multi readers
   - PostgreSqlSnapshotBase: postgres snapshot
   - SnapshotModeConstants: add some snapshot mode about postgres
   
   ### Verifying this change
   - [x] This change added tests and can be verified as follows:
    - add the Unit test of PostgreSqlReader in TestPostgresReader
    - add the Unit test of PostgreSqlSnapshotBase in TestPostgresOffsetManager
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] dockerzhang merged pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
dockerzhang merged PR #5367:
URL: https://github.com/apache/inlong/pull/5367


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r944140505


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestPostgresOffsetManager.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.inlong.agent.plugin.AgentBaseTestsHelper;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSqlSnapshotBase;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class TestPostgresOffsetManager {

Review Comment:
   OK, fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r944140614


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSqlSource.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Source;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * PostgreSql source, split PostgreSql source job into multi readers
+ */
+public class PostgreSqlSource implements Source {

Review Comment:
   OK, fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943186318


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSqlSource.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Source;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * PostgreSql source, split PostgreSql source job into multi readers
+ */
+public class PostgreSqlSource implements Source {

Review Comment:
   Do you mean `PostgreSQLSource`, beacause this class implements `Source` interface



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r942248082


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/SnapshotModeConstants.java:
##########
@@ -27,4 +27,12 @@ public class SnapshotModeConstants {
     public static final String EARLIEST_OFFSET = "never";
 
     public static final String SPECIFIC_OFFSETS = "schema_only_recovery";
+
+    public static final String ALWAYS = "always";

Review Comment:
   OK, i will move it to independent class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943534133


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestPostgresOffsetManager.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.inlong.agent.plugin.AgentBaseTestsHelper;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSqlSnapshotBase;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class TestPostgresOffsetManager {

Review Comment:
   Suggest changing the class name to `PostgreSQLOffsetManagerTest`, and adding some Java doc for all new classes, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943063074


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/snapshot/PostgreSqlSnapshotBase.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.snapshot;
+
+import org.apache.inlong.agent.utils.ThreadUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.FileOutputStream;
+import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.OutputStream;
+
+import java.util.Base64;
+
+public class PostgreSqlSnapshotBase implements SnapshotBase {
+
+    public static final int BUFFER_SIZE = 1024;
+    public static final int START_OFFSET = 0;
+    private static final Logger log = LoggerFactory.getLogger(PostgreSqlSnapshotBase.class);
+    private final Base64.Decoder decoder = Base64.getDecoder();
+    private final Base64.Encoder encoder = Base64.getEncoder();
+    private File file;
+    private byte[] offset;
+
+    public PostgreSqlSnapshotBase(String filePath) {
+        file = new File(filePath);
+    }
+
+    @Override
+    public String getSnapshot() {
+        load();
+        return encoder.encodeToString(offset);
+    }
+
+    @Override
+    public void close() {
+
+    }
+
+    private void load() {
+        try {
+            if (!file.exists()) {
+                // if parentDir not exist, create first
+                File parentDir = file.getParentFile();
+                if (parentDir == null) {
+                    log.info("no parent dir, file:{}",file.getAbsolutePath());

Review Comment:
   Please add one blank after `,`, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r942248378


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/reader/PostgreSqlReader.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.reader;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import io.debezium.connector.postgresql.PostgresConnector;
+import io.debezium.engine.ChangeEvent;
+import io.debezium.engine.DebeziumEngine;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.AgentConstants;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.message.DefaultMessage;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSqlSnapshotBase;
+import org.apache.inlong.agent.plugin.utils.InLongFileOffsetBackingStore;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.apache.inlong.agent.pojo.DebeziumOffset;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.apache.inlong.agent.utils.DebeziumOffsetSerializer;
+import org.apache.kafka.connect.storage.FileOffsetBackingStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_MAP_CAPACITY;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_KEY_DATA;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * read PostgreSql data
+ */
+public class PostgreSqlReader extends AbstractReader {
+
+
+    public static final String COMPONENT_NAME = "PostgreSqlReader";
+    public static final String JOB_DATABASE_USER = "job.postgreSqlJob.user";

Review Comment:
   OK



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] dockerzhang commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943349303


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSqlSource.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Source;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * PostgreSql source, split PostgreSql source job into multi readers
+ */
+public class PostgreSqlSource implements Source {

Review Comment:
   `PostgreSQLSource` looks good to me. Maybe we can `extends AbstractSource` instead of `implements Source` for referring to `KafkaSource` and `BinlogSource`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r945263442


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/PostgresSnapshotModeConstants.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.constant;
+
+/**
+ * Constants of job fetcher Postgressnapshot mode
+ */
+public class PostgresSnapshotModeConstants {

Review Comment:
   Suggest changing to `PostgreSQLConstants`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] dockerzhang commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r942221729


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/SnapshotModeConstants.java:
##########
@@ -27,4 +27,12 @@ public class SnapshotModeConstants {
     public static final String EARLIEST_OFFSET = "never";
 
     public static final String SPECIFIC_OFFSETS = "schema_only_recovery";
+
+    public static final String ALWAYS = "always";

Review Comment:
   if these `Constants` only used for PostgreSQL, it's better to be added in a independent class.



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/reader/PostgreSqlReader.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.reader;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import io.debezium.connector.postgresql.PostgresConnector;
+import io.debezium.engine.ChangeEvent;
+import io.debezium.engine.DebeziumEngine;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.AgentConstants;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.message.DefaultMessage;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSqlSnapshotBase;
+import org.apache.inlong.agent.plugin.utils.InLongFileOffsetBackingStore;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.apache.inlong.agent.pojo.DebeziumOffset;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.apache.inlong.agent.utils.DebeziumOffsetSerializer;
+import org.apache.kafka.connect.storage.FileOffsetBackingStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_MAP_CAPACITY;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_KEY_DATA;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * read PostgreSql data
+ */
+public class PostgreSqlReader extends AbstractReader {
+
+
+    public static final String COMPONENT_NAME = "PostgreSqlReader";
+    public static final String JOB_DATABASE_USER = "job.postgreSqlJob.user";

Review Comment:
   the following variables can be accessed out of the class; it's better to replace `DATABASE` with `PG` or `POSTGRESQL` to distinguish them.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r945265858


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLReaderTest.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSQLReader;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class PostgreSQLReaderTest {

Review Comment:
   I see that most Test class without Java doc, do I need to add Java doc in test class?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r945261329


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLReaderTest.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSQLReader;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class PostgreSQLReaderTest {
+    private static Gson gson = new Gson();

Review Comment:
   Suggested changing to `private static final Gson GSON = new Gson();`.



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/reader/PostgreSQLReader.java:
##########
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.reader;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import io.debezium.connector.postgresql.PostgresConnector;
+import io.debezium.engine.ChangeEvent;
+import io.debezium.engine.DebeziumEngine;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.AgentConstants;
+import org.apache.inlong.agent.constant.PostgresSnapshotModeConstants;
+import org.apache.inlong.agent.message.DefaultMessage;
+import org.apache.inlong.agent.metrics.AgentMetricItem;
+import org.apache.inlong.agent.metrics.audit.AuditUtils;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSQLSnapshotBase;
+import org.apache.inlong.agent.plugin.utils.InLongFileOffsetBackingStore;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.apache.inlong.agent.pojo.DebeziumOffset;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.apache.inlong.agent.utils.DebeziumOffsetSerializer;
+import org.apache.kafka.connect.storage.FileOffsetBackingStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_MAP_CAPACITY;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_KEY_DATA;
+
+/**
+ * read PostgreSql data
+ */
+public class PostgreSQLReader extends AbstractReader {
+
+
+    public static final String COMPONENT_NAME = "PostgreSQLReader";
+    public static final String JOB_POSTGRESQL_USER = "job.postgreSqlJob.user";
+    public static final String JOB_DATABASE_PASSWORD = "job.postgreSqlJob.password";
+    public static final String JOB_DATABASE_HOSTNAME = "job.postgreSqlJob.hostname";
+    public static final String JOB_DATABASE_PORT = "job.postgreSqlJob.port";
+    public static final String JOB_DATABASE_STORE_OFFSET_INTERVAL_MS = "job.postgreSqljob.offset.intervalMs";
+    public static final String JOB_DATABASE_STORE_HISTORY_FILENAME = "job.postgreSqljob.history.filename";
+    public static final String JOB_DATABASE_SNAPSHOT_MODE = "job.postgreSqljob.snapshot.mode";
+    public static final String JOB_DATABASE_QUEUE_SIZE = "job.postgreSqljob.queueSize";
+    public static final String JOB_DATABASE_OFFSETS = "job.postgreSqljob.offsets";
+    public static final String JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE = "job.postgreSqljob.offset.specificOffsetFile";
+    public static final String JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_POS = "job.postgreSqljob.offset.specificOffsetPos";
+    public static final String JOB_DATABASE_DBNAME = "job.postgreSqljob.dbname";
+    public static final String JOB_DATABASE_SERVER_NAME = "job.postgreSqljob.servername";
+    public static final String JOB_DATABASE_PLUGIN_NAME = "job.postgreSqljob.pluginname";
+    private static final Gson gson = new Gson();
+    private static final Logger LOGGER = LoggerFactory.getLogger(PostgreSQLReader.class);
+    private String userName;
+    private String password;
+    private String hostName;
+    private String port;
+    private String offsetFlushIntervalMs;
+    private String offsetStoreFileName;
+    private String snapshotMode;
+    private String instanceId;
+    private String offset;
+    private String specificOffsetFile;
+    private String specificOffsetPos;
+    private String dbName;
+    private String pluginName;
+    private String serverName;
+    private PostgreSQLSnapshotBase postgreSqlSnapshot;
+    private boolean finished = false;
+    private ExecutorService executor;
+    private final AgentConfiguration agentConf = AgentConfiguration.getAgentConf();
+
+    /**
+     * pair.left : table name
+     * pair.right : actual data
+     */
+    private LinkedBlockingQueue<Pair<String,String>> postgreSqlMessageQueue;
+
+    private JobProfile jobProfile;
+    private boolean destroyed = false;
+    protected AgentMetricItem readerMetric;
+
+    public PostgreSQLReader() {
+    }
+
+    @Override
+    public Message read() {
+        if (!postgreSqlMessageQueue.isEmpty()) {
+            readerMetric.pluginReadCount.incrementAndGet();
+            return getPostgreSqlMessage();
+        } else {
+            return null;
+        }
+    }
+
+    private DefaultMessage getPostgreSqlMessage() {
+        Pair<String, String> message = postgreSqlMessageQueue.poll();
+        Map<String,String> header = new HashMap<>(DEFAULT_MAP_CAPACITY);
+        header.put(PROXY_KEY_DATA,message.getKey());
+        return new DefaultMessage(message.getValue().getBytes(StandardCharsets.UTF_8),header);
+    }
+
+    @Override
+    public void init(JobProfile jobConf) {
+        jobProfile = jobConf;
+        LOGGER.info("init postgreSql reader with jobConf {}",jobConf.toJsonStr());
+        userName = jobConf.get(JOB_POSTGRESQL_USER);
+        password = jobConf.get(JOB_DATABASE_PASSWORD);
+        hostName = jobConf.get(JOB_DATABASE_HOSTNAME);
+        port = jobConf.get(JOB_DATABASE_PORT);
+        dbName = jobConf.get(JOB_DATABASE_DBNAME);
+        serverName = jobConf.get(JOB_DATABASE_SERVER_NAME);
+        pluginName = jobConf.get(JOB_DATABASE_PLUGIN_NAME,"pgoutput");
+        offsetFlushIntervalMs = jobConf.get(JOB_DATABASE_STORE_OFFSET_INTERVAL_MS,"100000");
+        offsetStoreFileName = jobConf.get(JOB_DATABASE_STORE_HISTORY_FILENAME,
+                tryToInitAndGetHistoryPath()) + "/offset.dat" + jobConf.getInstanceId();
+        snapshotMode = jobConf.get(JOB_DATABASE_SNAPSHOT_MODE,"");
+        postgreSqlMessageQueue = new LinkedBlockingQueue<>(jobConf.getInt(JOB_DATABASE_QUEUE_SIZE, 1000));
+        instanceId = jobConf.getInstanceId();
+        finished = false;
+
+        offset = jobConf.get(JOB_DATABASE_OFFSETS,"");
+        specificOffsetFile = jobConf.get(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE,"");
+        specificOffsetPos = jobConf.get(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_POS,"-1");
+        postgreSqlSnapshot = new PostgreSQLSnapshotBase(offsetStoreFileName);
+        postgreSqlSnapshot.save(offset);
+
+        Properties props = getEngineProps();
+
+        DebeziumEngine<ChangeEvent<String, String>> engine = DebeziumEngine.create(
+                io.debezium.engine.format.Json.class)
+                .using(props)
+                .notifying((records,committer) -> {
+                    try {
+                        for (ChangeEvent<String, String> record:records) {
+                            DebeziumFormat debeziumFormat = gson
+                                    .fromJson(record.value(),DebeziumFormat.class);
+                            postgreSqlMessageQueue.put(Pair.of(debeziumFormat.getSource().getTable(),record.value()));
+                            committer.markProcessed(record);
+                        }
+                        committer.markBatchFinished();
+                        AuditUtils.add(AuditUtils.AUDIT_ID_AGENT_READ_SUCCESS, inlongGroupId, inlongStreamId,
+                                System.currentTimeMillis(), records.size());
+                        readerMetric.pluginReadCount.addAndGet(records.size());
+                    } catch (Exception e) {
+                        readerMetric.pluginReadFailCount.addAndGet(records.size());
+                        LOGGER.error("parse binlog message error",e);
+                    }
+                })
+                .using((success, message, error) -> {
+                    if (!success) {
+                        LOGGER.error("postgreslog job with jobConf {} has error {}",
+                                jobConf.getInstanceId(),message,error);
+                    }
+                }).build();
+
+        executor = Executors.newSingleThreadExecutor();
+        executor.execute(engine);
+
+        LOGGER.info("get initial snapshot of job {}, snapshot {}",
+                jobConf.getInstanceId(),getSnapshot());
+
+    }
+
+    private String tryToInitAndGetHistoryPath() {
+        String historyPath = agentConf.get(
+                AgentConstants.AGENT_HISTORY_PATH, AgentConstants.DEFAULT_AGENT_HISTORY_PATH);
+        String parentPath = agentConf.get(
+                AgentConstants.AGENT_HOME, AgentConstants.DEFAULT_AGENT_HOME);
+        return AgentUtils.makeDirsIfNotExist(historyPath, parentPath).getAbsolutePath();
+    }
+
+    private Properties getEngineProps() {
+        Properties props = new Properties();
+
+        props.setProperty("name","engine" + instanceId);
+        props.setProperty("connector.class", PostgresConnector.class.getCanonicalName());
+        props.setProperty("database.server.name", serverName);
+        props.setProperty("plugin.name",pluginName);
+        props.setProperty("database.hostname", hostName);
+        props.setProperty("database.port", port);
+        props.setProperty("database.user", userName);
+        props.setProperty("database.dbname",dbName);
+        props.setProperty("database.password", password);
+
+        props.setProperty("offset.flush.interval.ms",offsetFlushIntervalMs);
+        props.setProperty("database.snapshot.mode",snapshotMode);
+        props.setProperty("key.converter.schemas.enable", "false");
+        props.setProperty("value.converter.schemas.enable", "false");
+        props.setProperty("snapshot.mode", snapshotMode);
+        props.setProperty("offset.storage.file.filename", offsetStoreFileName);
+        if (PostgresSnapshotModeConstants.CUSTOM.equals(snapshotMode)) {
+            props.setProperty("offset.storage", InLongFileOffsetBackingStore.class.getCanonicalName());
+            props.setProperty(InLongFileOffsetBackingStore.OFFSET_STATE_VALUE,serializeOffset());
+        } else {
+            props.setProperty("offset.storage", FileOffsetBackingStore.class.getCanonicalName());
+        }
+        props.setProperty("tombstones.on.delete", "false");
+        props.setProperty("converters", "datetime");
+        props.setProperty("datetime.type", "org.apache.inlong.agent.plugin.utils.BinlogTimeConverter");
+        props.setProperty("datetime.format.date", "yyyy-MM-dd");
+        props.setProperty("datetime.format.time", "HH:mm:ss");
+        props.setProperty("datetime.format.datetime", "yyyy-MM-dd HH:mm:ss");
+        props.setProperty("datetime.format.timestamp", "yyyy-MM-dd HH:mm:ss");
+
+        LOGGER.info("postgreslog job {} start with props {}",jobProfile.getInstanceId(),props);
+        return props;
+    }
+
+    private String serializeOffset() {
+        Map<String, Object> sourceOffset = new HashMap<>();
+        Preconditions.checkNotNull(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE,
+                JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE + "shouldn't be null");
+        sourceOffset.put("file",specificOffsetFile);

Review Comment:
   It is recommended to double-check and add a space after the comma to increase the readability of the code.



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/snapshot/PostgreSQLSnapshotBase.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.snapshot;
+
+import org.apache.inlong.agent.utils.ThreadUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.FileOutputStream;
+import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.OutputStream;
+
+import java.util.Base64;
+
+/**
+ * PostgreSQL Snapshot
+ */
+public class PostgreSQLSnapshotBase implements SnapshotBase {
+
+    public static final int BUFFER_SIZE = 1024;
+    public static final int START_OFFSET = 0;
+    private static final Logger log = LoggerFactory.getLogger(PostgreSQLSnapshotBase.class);

Review Comment:
   It is suggested to change the `static final` variable to UPPER_CASE.



##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLReaderTest.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSQLReader;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class PostgreSQLReaderTest {

Review Comment:
   Please add Java doc for new classes.



##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLOffsetManagerTest.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.inlong.agent.plugin.AgentBaseTestsHelper;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSQLSnapshotBase;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class PostgreSQLOffsetManagerTest {

Review Comment:
   Java doc, please.



##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLReaderTest.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSQLReader;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class PostgreSQLReaderTest {
+    private static Gson gson = new Gson();
+
+    @Test
+    public void testDebeziumFormat() {
+        String debeziumJson = "{\n"
+                + "    \"before\": null,\n"
+                + "    \"after\": {\n"
+                + "      \"id\": 1004,\n"
+                + "      \"first_name\": \"Anne\",\n"
+                + "      \"last_name\": \"Kretchmar\",\n"
+                + "      \"email\": \"annek@noanswer.org\"\n"
+                + "    },\n"
+                + "    \"source\": {\n"
+                + "      \"version\": \"12\",\n"
+                + "      \"name\": \"myserver\",\n"
+                + "      \"ts_sec\": 0,\n"
+                + "      \"gtid\": null,\n"
+                + "      \"file\": \"000000010000000000000001\",\n"
+                + "      \"row\": 0,\n"
+                + "      \"snapshot\": true,\n"
+                + "      \"thread\": null,\n"
+                + "      \"db\": \"postgres\",\n"
+                + "      \"table\": \"customers\"\n"
+                + "    },\n"
+                + "    \"op\": \"r\",\n"
+                + "    \"ts_ms\": 1486500577691\n"
+                + "  }";
+        DebeziumFormat debeziumFormat = gson
+                .fromJson(debeziumJson, DebeziumFormat.class);
+        Assert.assertEquals("customers",debeziumFormat.getSource().getTable());
+        Assert.assertEquals("true",debeziumFormat.getSource().getSnapshot());
+    }
+
+//    @Test

Review Comment:
   Why comment out this Test annotation?



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/snapshot/PostgreSQLSnapshotBase.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.snapshot;
+
+import org.apache.inlong.agent.utils.ThreadUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.FileOutputStream;
+import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.OutputStream;
+
+import java.util.Base64;
+
+/**
+ * PostgreSQL Snapshot
+ */
+public class PostgreSQLSnapshotBase implements SnapshotBase {
+
+    public static final int BUFFER_SIZE = 1024;
+    public static final int START_OFFSET = 0;
+    private static final Logger log = LoggerFactory.getLogger(PostgreSQLSnapshotBase.class);
+    private final Base64.Decoder decoder = Base64.getDecoder();
+    private final Base64.Encoder encoder = Base64.getEncoder();
+    private File file;
+    private byte[] offset;
+
+    public PostgreSQLSnapshotBase(String filePath) {
+        file = new File(filePath);
+    }
+
+    @Override
+    public String getSnapshot() {
+        load();
+        return encoder.encodeToString(offset);
+    }
+
+    @Override
+    public void close() {
+
+    }
+
+    private void load() {
+        try {
+            if (!file.exists()) {
+                // if parentDir not exist, create first
+                File parentDir = file.getParentFile();
+                if (parentDir == null) {
+                    log.info("no parent dir, file:{}", file.getAbsolutePath());
+                    return;
+                }
+                if (!parentDir.exists()) {
+                    boolean success = parentDir.mkdir();
+                    log.info("create dir {} result {}", parentDir, success);
+                }
+                file.createNewFile();
+            }
+            FileInputStream fis = new FileInputStream(file);
+            BufferedInputStream inputStream = new BufferedInputStream(fis);
+            ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+            int len;
+            byte[] buf = new byte[BUFFER_SIZE];
+            while ((len = inputStream.read(buf)) != -1) {
+                outputStream.write(buf, START_OFFSET, len);
+            }
+            offset = outputStream.toByteArray();
+            inputStream.close();
+            outputStream.close();
+        } catch (Throwable ex) {
+            log.error("load postgreSql WAL log error", ex);
+            ThreadUtils.threadThrowableHandler(Thread.currentThread(), ex);
+        }
+    }
+
+    /**
+     * save binlog offset to local file
+     */
+    public void save(String snapShot) {

Review Comment:
   The `snapshot` is one word, so just change the `snapShot` to `snapshot`.



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/reader/PostgreSQLReader.java:
##########
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources.reader;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import io.debezium.connector.postgresql.PostgresConnector;
+import io.debezium.engine.ChangeEvent;
+import io.debezium.engine.DebeziumEngine;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.AgentConstants;
+import org.apache.inlong.agent.constant.PostgresSnapshotModeConstants;
+import org.apache.inlong.agent.message.DefaultMessage;
+import org.apache.inlong.agent.metrics.AgentMetricItem;
+import org.apache.inlong.agent.metrics.audit.AuditUtils;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.snapshot.PostgreSQLSnapshotBase;
+import org.apache.inlong.agent.plugin.utils.InLongFileOffsetBackingStore;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.apache.inlong.agent.pojo.DebeziumOffset;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.apache.inlong.agent.utils.DebeziumOffsetSerializer;
+import org.apache.kafka.connect.storage.FileOffsetBackingStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_MAP_CAPACITY;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_KEY_DATA;
+
+/**
+ * read PostgreSql data
+ */
+public class PostgreSQLReader extends AbstractReader {
+
+
+    public static final String COMPONENT_NAME = "PostgreSQLReader";
+    public static final String JOB_POSTGRESQL_USER = "job.postgreSqlJob.user";
+    public static final String JOB_DATABASE_PASSWORD = "job.postgreSqlJob.password";
+    public static final String JOB_DATABASE_HOSTNAME = "job.postgreSqlJob.hostname";
+    public static final String JOB_DATABASE_PORT = "job.postgreSqlJob.port";
+    public static final String JOB_DATABASE_STORE_OFFSET_INTERVAL_MS = "job.postgreSqljob.offset.intervalMs";
+    public static final String JOB_DATABASE_STORE_HISTORY_FILENAME = "job.postgreSqljob.history.filename";
+    public static final String JOB_DATABASE_SNAPSHOT_MODE = "job.postgreSqljob.snapshot.mode";
+    public static final String JOB_DATABASE_QUEUE_SIZE = "job.postgreSqljob.queueSize";
+    public static final String JOB_DATABASE_OFFSETS = "job.postgreSqljob.offsets";
+    public static final String JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE = "job.postgreSqljob.offset.specificOffsetFile";
+    public static final String JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_POS = "job.postgreSqljob.offset.specificOffsetPos";
+    public static final String JOB_DATABASE_DBNAME = "job.postgreSqljob.dbname";
+    public static final String JOB_DATABASE_SERVER_NAME = "job.postgreSqljob.servername";
+    public static final String JOB_DATABASE_PLUGIN_NAME = "job.postgreSqljob.pluginname";
+    private static final Gson gson = new Gson();
+    private static final Logger LOGGER = LoggerFactory.getLogger(PostgreSQLReader.class);
+    private String userName;
+    private String password;
+    private String hostName;
+    private String port;
+    private String offsetFlushIntervalMs;
+    private String offsetStoreFileName;
+    private String snapshotMode;
+    private String instanceId;
+    private String offset;
+    private String specificOffsetFile;
+    private String specificOffsetPos;
+    private String dbName;
+    private String pluginName;
+    private String serverName;
+    private PostgreSQLSnapshotBase postgreSqlSnapshot;
+    private boolean finished = false;
+    private ExecutorService executor;
+    private final AgentConfiguration agentConf = AgentConfiguration.getAgentConf();
+
+    /**
+     * pair.left : table name
+     * pair.right : actual data
+     */
+    private LinkedBlockingQueue<Pair<String,String>> postgreSqlMessageQueue;
+
+    private JobProfile jobProfile;
+    private boolean destroyed = false;
+    protected AgentMetricItem readerMetric;
+
+    public PostgreSQLReader() {
+    }
+
+    @Override
+    public Message read() {
+        if (!postgreSqlMessageQueue.isEmpty()) {
+            readerMetric.pluginReadCount.incrementAndGet();
+            return getPostgreSqlMessage();
+        } else {
+            return null;
+        }
+    }
+
+    private DefaultMessage getPostgreSqlMessage() {
+        Pair<String, String> message = postgreSqlMessageQueue.poll();
+        Map<String,String> header = new HashMap<>(DEFAULT_MAP_CAPACITY);
+        header.put(PROXY_KEY_DATA,message.getKey());
+        return new DefaultMessage(message.getValue().getBytes(StandardCharsets.UTF_8),header);
+    }
+
+    @Override
+    public void init(JobProfile jobConf) {
+        jobProfile = jobConf;
+        LOGGER.info("init postgreSql reader with jobConf {}",jobConf.toJsonStr());
+        userName = jobConf.get(JOB_POSTGRESQL_USER);
+        password = jobConf.get(JOB_DATABASE_PASSWORD);
+        hostName = jobConf.get(JOB_DATABASE_HOSTNAME);
+        port = jobConf.get(JOB_DATABASE_PORT);
+        dbName = jobConf.get(JOB_DATABASE_DBNAME);
+        serverName = jobConf.get(JOB_DATABASE_SERVER_NAME);
+        pluginName = jobConf.get(JOB_DATABASE_PLUGIN_NAME,"pgoutput");
+        offsetFlushIntervalMs = jobConf.get(JOB_DATABASE_STORE_OFFSET_INTERVAL_MS,"100000");
+        offsetStoreFileName = jobConf.get(JOB_DATABASE_STORE_HISTORY_FILENAME,
+                tryToInitAndGetHistoryPath()) + "/offset.dat" + jobConf.getInstanceId();
+        snapshotMode = jobConf.get(JOB_DATABASE_SNAPSHOT_MODE,"");
+        postgreSqlMessageQueue = new LinkedBlockingQueue<>(jobConf.getInt(JOB_DATABASE_QUEUE_SIZE, 1000));
+        instanceId = jobConf.getInstanceId();
+        finished = false;
+
+        offset = jobConf.get(JOB_DATABASE_OFFSETS,"");
+        specificOffsetFile = jobConf.get(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE,"");
+        specificOffsetPos = jobConf.get(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_POS,"-1");
+        postgreSqlSnapshot = new PostgreSQLSnapshotBase(offsetStoreFileName);
+        postgreSqlSnapshot.save(offset);
+
+        Properties props = getEngineProps();
+
+        DebeziumEngine<ChangeEvent<String, String>> engine = DebeziumEngine.create(
+                io.debezium.engine.format.Json.class)
+                .using(props)
+                .notifying((records,committer) -> {
+                    try {
+                        for (ChangeEvent<String, String> record:records) {
+                            DebeziumFormat debeziumFormat = gson
+                                    .fromJson(record.value(),DebeziumFormat.class);
+                            postgreSqlMessageQueue.put(Pair.of(debeziumFormat.getSource().getTable(),record.value()));
+                            committer.markProcessed(record);
+                        }
+                        committer.markBatchFinished();
+                        AuditUtils.add(AuditUtils.AUDIT_ID_AGENT_READ_SUCCESS, inlongGroupId, inlongStreamId,
+                                System.currentTimeMillis(), records.size());
+                        readerMetric.pluginReadCount.addAndGet(records.size());
+                    } catch (Exception e) {
+                        readerMetric.pluginReadFailCount.addAndGet(records.size());
+                        LOGGER.error("parse binlog message error",e);
+                    }
+                })
+                .using((success, message, error) -> {
+                    if (!success) {
+                        LOGGER.error("postgreslog job with jobConf {} has error {}",
+                                jobConf.getInstanceId(),message,error);
+                    }
+                }).build();
+
+        executor = Executors.newSingleThreadExecutor();
+        executor.execute(engine);
+
+        LOGGER.info("get initial snapshot of job {}, snapshot {}",
+                jobConf.getInstanceId(),getSnapshot());
+
+    }
+
+    private String tryToInitAndGetHistoryPath() {
+        String historyPath = agentConf.get(
+                AgentConstants.AGENT_HISTORY_PATH, AgentConstants.DEFAULT_AGENT_HISTORY_PATH);
+        String parentPath = agentConf.get(
+                AgentConstants.AGENT_HOME, AgentConstants.DEFAULT_AGENT_HOME);
+        return AgentUtils.makeDirsIfNotExist(historyPath, parentPath).getAbsolutePath();
+    }
+
+    private Properties getEngineProps() {
+        Properties props = new Properties();
+
+        props.setProperty("name","engine" + instanceId);
+        props.setProperty("connector.class", PostgresConnector.class.getCanonicalName());
+        props.setProperty("database.server.name", serverName);
+        props.setProperty("plugin.name",pluginName);
+        props.setProperty("database.hostname", hostName);
+        props.setProperty("database.port", port);
+        props.setProperty("database.user", userName);
+        props.setProperty("database.dbname",dbName);
+        props.setProperty("database.password", password);
+
+        props.setProperty("offset.flush.interval.ms",offsetFlushIntervalMs);
+        props.setProperty("database.snapshot.mode",snapshotMode);
+        props.setProperty("key.converter.schemas.enable", "false");
+        props.setProperty("value.converter.schemas.enable", "false");
+        props.setProperty("snapshot.mode", snapshotMode);
+        props.setProperty("offset.storage.file.filename", offsetStoreFileName);
+        if (PostgresSnapshotModeConstants.CUSTOM.equals(snapshotMode)) {
+            props.setProperty("offset.storage", InLongFileOffsetBackingStore.class.getCanonicalName());
+            props.setProperty(InLongFileOffsetBackingStore.OFFSET_STATE_VALUE,serializeOffset());
+        } else {
+            props.setProperty("offset.storage", FileOffsetBackingStore.class.getCanonicalName());
+        }
+        props.setProperty("tombstones.on.delete", "false");
+        props.setProperty("converters", "datetime");
+        props.setProperty("datetime.type", "org.apache.inlong.agent.plugin.utils.BinlogTimeConverter");
+        props.setProperty("datetime.format.date", "yyyy-MM-dd");
+        props.setProperty("datetime.format.time", "HH:mm:ss");
+        props.setProperty("datetime.format.datetime", "yyyy-MM-dd HH:mm:ss");
+        props.setProperty("datetime.format.timestamp", "yyyy-MM-dd HH:mm:ss");
+
+        LOGGER.info("postgreslog job {} start with props {}",jobProfile.getInstanceId(),props);
+        return props;
+    }
+
+    private String serializeOffset() {
+        Map<String, Object> sourceOffset = new HashMap<>();
+        Preconditions.checkNotNull(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE,
+                JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE + "shouldn't be null");
+        sourceOffset.put("file",specificOffsetFile);
+        Preconditions.checkNotNull(JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_POS,
+                            JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_POS + "shouldn't be null");
+        sourceOffset.put("pos",specificOffsetPos);
+        DebeziumOffset specificOffset = new DebeziumOffset();
+        specificOffset.setSourceOffset(sourceOffset);
+        Map<String, String> sourcePartition = new HashMap<>();
+        sourcePartition.put("server",instanceId);
+        specificOffset.setSourcePartition(sourcePartition);
+        byte[] serializedOffset = new byte[0];
+        try {
+            serializedOffset = DebeziumOffsetSerializer.INSTANCE.serialize(specificOffset);
+        } catch (IOException e) {
+            LOGGER.error("serialize offset message error",e);
+        }
+        return new String(serializedOffset,StandardCharsets.UTF_8);
+    }
+
+    @Override
+    public void destroy() {
+        synchronized (this) {
+            if (!destroyed) {
+                executor.shutdownNow();
+                postgreSqlSnapshot.close();
+                destroyed = true;
+            }
+        }
+    }
+
+    @Override
+    public boolean isFinished() {
+        return finished;
+    }
+
+    @Override
+    public String getReadSource() {
+        return instanceId;
+    }
+
+    @Override
+    public void setReadTimeout(long mill) {
+        return;
+    }
+
+    @Override
+    public void setWaitMillisecond(long millis) {
+        return;
+    }
+
+    @Override
+    public String getSnapshot() {
+        if (postgreSqlSnapshot != null) {
+            return postgreSqlSnapshot.getSnapshot();
+        } else {
+            return "";

Review Comment:
   Unnecessary `else` statement.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r946275388


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLReaderTest.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSQLReader;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+/**
+ * test postgres reader
+ */
+public class PostgreSQLReaderTest {
+    private static Gson GSON = new Gson();
+
+    @Test
+    public void testDebeziumFormat() {
+        String debeziumJson = "{\n"
+                + "    \"before\": null,\n"
+                + "    \"after\": {\n"
+                + "      \"id\": 1004,\n"
+                + "      \"first_name\": \"Anne\",\n"
+                + "      \"last_name\": \"Kretchmar\",\n"
+                + "      \"email\": \"annek@noanswer.org\"\n"
+                + "    },\n"
+                + "    \"source\": {\n"
+                + "      \"version\": \"12\",\n"
+                + "      \"name\": \"myserver\",\n"
+                + "      \"ts_sec\": 0,\n"
+                + "      \"gtid\": null,\n"
+                + "      \"file\": \"000000010000000000000001\",\n"
+                + "      \"row\": 0,\n"
+                + "      \"snapshot\": true,\n"
+                + "      \"thread\": null,\n"
+                + "      \"db\": \"postgres\",\n"
+                + "      \"table\": \"customers\"\n"
+                + "    },\n"
+                + "    \"op\": \"r\",\n"
+                + "    \"ts_ms\": 1486500577691\n"
+                + "  }";
+        DebeziumFormat debeziumFormat = GSON
+                .fromJson(debeziumJson, DebeziumFormat.class);
+        Assert.assertEquals("customers", debeziumFormat.getSource().getTable());
+        Assert.assertEquals("true", debeziumFormat.getSource().getSnapshot());
+    }
+
+//    @Test
+    public void postgresLoadTest() {
+        JobProfile jobProfile = new JobProfile();
+        jobProfile.set(PostgreSQLReader.JOB_POSTGRESQL_USER, "postgres");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_SERVER_NAME, "postgres");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_PLUGIN_NAME, "pgoutput");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_PASSWORD, "123456");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_HOSTNAME, "localhost");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_PORT, "5432");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_OFFSET_SPECIFIC_OFFSET_FILE, "000000010000000000000001");
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_SNAPSHOT_MODE, SnapshotModeConstants.INITIAL);
+        jobProfile.set(PostgreSQLReader.JOB_DATABASE_DBNAME, "postgres");
+        jobProfile.set("job.instance.id", "_1");
+        jobProfile.set(PROXY_INLONG_GROUP_ID, "groupid");
+        jobProfile.set(PROXY_INLONG_STREAM_ID, "streamid");
+        PostgreSQLReader postgreSqlReader = new PostgreSQLReader();
+        postgreSqlReader.init(jobProfile);
+        while (true) {
+            Message message = postgreSqlReader.read();
+            if (message != null) {
+                System.out.println(message.toString());

Review Comment:
   Please not use the `System.out.println`, replace it with log framework.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] iamsee123 commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
iamsee123 commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r945265744


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/PostgreSQLReaderTest.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSQLReader;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class PostgreSQLReaderTest {
+    private static Gson gson = new Gson();
+
+    @Test
+    public void testDebeziumFormat() {
+        String debeziumJson = "{\n"
+                + "    \"before\": null,\n"
+                + "    \"after\": {\n"
+                + "      \"id\": 1004,\n"
+                + "      \"first_name\": \"Anne\",\n"
+                + "      \"last_name\": \"Kretchmar\",\n"
+                + "      \"email\": \"annek@noanswer.org\"\n"
+                + "    },\n"
+                + "    \"source\": {\n"
+                + "      \"version\": \"12\",\n"
+                + "      \"name\": \"myserver\",\n"
+                + "      \"ts_sec\": 0,\n"
+                + "      \"gtid\": null,\n"
+                + "      \"file\": \"000000010000000000000001\",\n"
+                + "      \"row\": 0,\n"
+                + "      \"snapshot\": true,\n"
+                + "      \"thread\": null,\n"
+                + "      \"db\": \"postgres\",\n"
+                + "      \"table\": \"customers\"\n"
+                + "    },\n"
+                + "    \"op\": \"r\",\n"
+                + "    \"ts_ms\": 1486500577691\n"
+                + "  }";
+        DebeziumFormat debeziumFormat = gson
+                .fromJson(debeziumJson, DebeziumFormat.class);
+        Assert.assertEquals("customers",debeziumFormat.getSource().getTable());
+        Assert.assertEquals("true",debeziumFormat.getSource().getSnapshot());
+    }
+
+//    @Test

Review Comment:
   Because the test is used for testing collect data from postgres in unit test it may cause failure in compile



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] EMsnap commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
EMsnap commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r945383892


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/PostgreSQLConstants.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.constant;
+
+/**
+ * Constants of job fetcher Postgressnapshot mode

Review Comment:
   Postgressnapshot -> Postgres Snapshot



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943062805


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestPostgresReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class TestPostgresReader {

Review Comment:
   Suggest changing the class name to `PostgreSQLReaderTest`.



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSqlSource.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Source;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * PostgreSql source, split PostgreSql source job into multi readers
+ */
+public class PostgreSqlSource implements Source {

Review Comment:
   Suggest changing the class name to `PostgreSQLReader`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943062962


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestPostgresReader.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constant.SnapshotModeConstants;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.pojo.DebeziumFormat;
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+
+public class TestPostgresReader {
+    private static Gson gson = new Gson();
+
+    @Test
+    public void testDebeziumFormat() {
+        String debeziumJson = "{\n"
+                + "    \"before\": null,\n"
+                + "    \"after\": {\n"
+                + "      \"id\": 1004,\n"
+                + "      \"first_name\": \"Anne\",\n"
+                + "      \"last_name\": \"Kretchmar\",\n"
+                + "      \"email\": \"annek@noanswer.org\"\n"
+                + "    },\n"
+                + "    \"source\": {\n"
+                + "      \"version\": \"12\",\n"
+                + "      \"name\": \"myserver\",\n"
+                + "      \"ts_sec\": 0,\n"
+                + "      \"gtid\": null,\n"
+                + "      \"file\": \"000000010000000000000001\",\n"
+                + "      \"row\": 0,\n"
+                + "      \"snapshot\": true,\n"
+                + "      \"thread\": null,\n"
+                + "      \"db\": \"postgres\",\n"
+                + "      \"table\": \"customers\"\n"
+                + "    },\n"
+                + "    \"op\": \"r\",\n"
+                + "    \"ts_ms\": 1486500577691\n"
+                + "  }";
+        DebeziumFormat debeziumFormat = gson
+                .fromJson(debeziumJson, DebeziumFormat.class);
+        Assert.assertEquals("customers",debeziumFormat.getSource().getTable());
+        Assert.assertEquals("true",debeziumFormat.getSource().getSnapshot());
+    }
+
+//    @Test
+    public void postgresLoadTest() {
+        JobProfile jobProfile = new JobProfile();
+        jobProfile.set(PostgreSqlReader.JOB_DATABASE_USER,"postgres");

Review Comment:
   Please add one blank after `,`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] healchow commented on a diff in pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5367:
URL: https://github.com/apache/inlong/pull/5367#discussion_r943537679


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSqlSource.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.inlong.agent.plugin.sources;
+
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Source;
+import org.apache.inlong.agent.plugin.sources.reader.PostgreSqlReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_GROUP_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_STREAM_ID;
+import static org.apache.inlong.agent.constant.AgentConstants.GLOBAL_METRICS;
+
+/**
+ * PostgreSql source, split PostgreSql source job into multi readers
+ */
+public class PostgreSqlSource implements Source {

Review Comment:
   Sorry I didn't describe it clearly.
   
   My concern is that `PostgreSQL` is the official name, and `Postgres`, `PostgreSql`, etc. are not official names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] dockerzhang commented on pull request #5367: [INLONG-5046][Agent] Support collect data from PostgreSQL

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on PR #5367:
URL: https://github.com/apache/inlong/pull/5367#issuecomment-1212772125

   @haibo-duan PTAL, thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org