You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/08/17 09:01:57 UTC

[shardingsphere] branch master updated: Implement PostgreSQL authentication for proxy (#5978) (#6860)

This is an automated email from the ASF dual-hosted git repository.

zhangyonglun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new ca220d0  Implement PostgreSQL authentication for proxy (#5978) (#6860)
ca220d0 is described below

commit ca220d0b447db4a994e4972b397b589b1df1c1c9
Author: sandynz <42...@users.noreply.github.com>
AuthorDate: Mon Aug 17 17:01:39 2020 +0800

    Implement PostgreSQL authentication for proxy (#5978) (#6860)
    
    * Implement PostgreSQL authentication for proxy (#5978)
    
    * Fix rat
    
    * Implement PostgreSQL authentication for proxy
    
    * Implement PostgreSQL authentication for proxy - follow review suggestion
---
 .../postgresql/constant/PostgreSQLErrorCode.java   | 74 +++++++++++++++++++
 .../PostgreSQLAuthenticationMD5PasswordPacket.java | 46 ++++++++++++
 .../handshake/PostgreSQLPasswordMessagePacket.java | 46 ++++++++++++
 .../handshake/PostgreSQLRandomGenerator.java       | 53 ++++++++++++++
 .../handshake/PostgreSQLRandomGeneratorTest.java   | 34 +++++++++
 .../auth/PostgreSQLAuthenticationEngine.java       | 85 ++++++++++++++++++----
 .../auth/PostgreSQLAuthenticationHandler.java      | 76 +++++++++++++++++++
 .../postgresql/auth/PostgreSQLLoginResult.java     | 37 ++++++++++
 8 files changed, 435 insertions(+), 16 deletions(-)

diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/constant/PostgreSQLErrorCode.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/constant/PostgreSQLErrorCode.java
new file mode 100644
index 0000000..5fab09d
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/constant/PostgreSQLErrorCode.java
@@ -0,0 +1,74 @@
+/*
+ * 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.shardingsphere.db.protocol.postgresql.constant;
+
+/**
+ * PostgreSQL error code.
+ *
+ * @see <a href="https://www.postgresql.org/docs/12/errcodes-appendix.html">Appendix A. PostgreSQL Error Codes</a>
+ */
+public enum PostgreSQLErrorCode {
+    
+    SUCCESSFUL_COMPLETION("00000", "successful_completion"),
+    WARNING("01000", "warning"),
+    DYNAMIC_RESULT_SETS_RETURNED("0100C", "dynamic_result_sets_returned"),
+    IMPLICIT_ZERO_BIT_PADDING("01008", "implicit_zero_bit_padding"),
+    NULL_VALUE_ELIMINATED_IN_SET_FUNCTION("01003", "null_value_eliminated_in_set_function"),
+    PRIVILEGE_NOT_GRANTED("01007", "privilege_not_granted"),
+    PRIVILEGE_NOT_REVOKED("01006", "privilege_not_revoked"),
+    STRING_DATA_RIGHT_TRUNCATION("01004", "string_data_right_truncation"),
+    DEPRECATED_FEATURE("01P01", "deprecated_feature"),
+    CONNECTION_EXCEPTION("08000", "connection_exception"),
+    CONNECTION_DOES_NOT_EXIST("08003", "connection_does_not_exist"),
+    CONNECTION_FAILURE("08006", "connection_failure"),
+    SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION("08001", "sqlclient_unable_to_establish_sqlconnection"),
+    SQLSERVER_REJECTED_ESTABLISHMENT_OF_SQLCONNECTION("08004", "sqlserver_rejected_establishment_of_sqlconnection"),
+    TRANSACTION_RESOLUTION_UNKNOWN("08007", "transaction_resolution_unknown"),
+    PROTOCOL_VIOLATION("08P01", "protocol_violation"),
+    INVALID_AUTHORIZATION_SPECIFICATION("28000", "invalid_authorization_specification"),
+    INVALID_PASSWORD("28P01", "invalid_password"),
+    INVALID_CATALOG_NAME("3D000", "invalid_catalog_name"),
+    INVALID_SCHEMA_NAME("3F000", "invalid_schema_name"),;
+    
+    private final String errorCode;
+    
+    private final String conditionName;
+    
+    PostgreSQLErrorCode(final String errorCode, final String conditionName) {
+        this.errorCode = errorCode;
+        this.conditionName = conditionName;
+    }
+    
+    /**
+     * Get error code.
+     *
+     * @return error code
+     */
+    public String getErrorCode() {
+        return errorCode;
+    }
+    
+    /**
+     * Get condition name.
+     *
+     * @return condition name
+     */
+    public String getConditionName() {
+        return conditionName;
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLAuthenticationMD5PasswordPacket.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLAuthenticationMD5PasswordPacket.java
new file mode 100644
index 0000000..59f0d62
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLAuthenticationMD5PasswordPacket.java
@@ -0,0 +1,46 @@
+/*
+ * 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.shardingsphere.db.protocol.postgresql.packet.handshake;
+
+import lombok.Getter;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.PostgreSQLPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * AuthenticationMD5Password (backend) packet for PostgreSQL.
+ */
+public final class PostgreSQLAuthenticationMD5PasswordPacket implements PostgreSQLPacket {
+    
+    private static final int AUTH_REQ_MD5 = 5;
+    
+    private final byte[] md5Salt;
+    
+    @Getter
+    private final char messageType = PostgreSQLCommandPacketType.AUTHENTICATION_MD5_PASSWORD.getValue();
+    
+    public PostgreSQLAuthenticationMD5PasswordPacket(final byte[] md5Salt) {
+        this.md5Salt = md5Salt;
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload) {
+        payload.writeInt4(AUTH_REQ_MD5);
+        payload.writeBytes(md5Salt);
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLPasswordMessagePacket.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLPasswordMessagePacket.java
new file mode 100644
index 0000000..1ab4492
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLPasswordMessagePacket.java
@@ -0,0 +1,46 @@
+/*
+ * 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.shardingsphere.db.protocol.postgresql.packet.handshake;
+
+import lombok.Getter;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.PostgreSQLPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * PasswordMessage (frontend) packet for PostgreSQL.
+ */
+public final class PostgreSQLPasswordMessagePacket implements PostgreSQLPacket {
+    
+    @Getter
+    private final String md5Digest;
+    
+    public PostgreSQLPasswordMessagePacket(final PostgreSQLPacketPayload payload) {
+        payload.readInt4();
+        md5Digest = payload.readStringNul();
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload) {
+    }
+    
+    @Override
+    public char getMessageType() {
+        return PostgreSQLCommandPacketType.PASSWORD_MESSAGE.getValue();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLRandomGenerator.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLRandomGenerator.java
new file mode 100644
index 0000000..eb22060
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLRandomGenerator.java
@@ -0,0 +1,53 @@
+/*
+ * 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.shardingsphere.db.protocol.postgresql.packet.handshake;
+
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+
+/**
+ * Random generator for PostgreSQL.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class PostgreSQLRandomGenerator {
+    
+    private static final PostgreSQLRandomGenerator INSTANCE = new PostgreSQLRandomGenerator();
+    
+    /**
+     * Get instance.
+     *
+     * @return instance
+     */
+    public static PostgreSQLRandomGenerator getInstance() {
+        return INSTANCE;
+    }
+    
+    /**
+     * Generate random bytes.
+     *
+     * @param length length for generated random bytes.
+     * @return generated random bytes
+     */
+    public byte[] generateRandomBytes(final int length) {
+        byte[] result = new byte[length];
+        ThreadLocalRandom.current().nextBytes(result);
+        return result;
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLRandomGeneratorTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLRandomGeneratorTest.java
new file mode 100644
index 0000000..611e602
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/handshake/PostgreSQLRandomGeneratorTest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.shardingsphere.db.protocol.postgresql.packet.handshake;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class PostgreSQLRandomGeneratorTest {
+    
+    @Test
+    public void t() {
+        PostgreSQLRandomGenerator generator = PostgreSQLRandomGenerator.getInstance();
+        for (int i = 1; i < 13; i++) {
+            assertThat(generator.generateRandomBytes(i).length, is(i));
+        }
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationEngine.java
index f534ddf..0dc0ea4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationEngine.java
@@ -19,13 +19,18 @@ package org.apache.shardingsphere.proxy.frontend.postgresql.auth;
 
 import com.google.common.base.Strings;
 import io.netty.channel.ChannelHandlerContext;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.shardingsphere.db.protocol.payload.PacketPayload;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLErrorCode;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.BinaryStatementRegistry;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.generic.PostgreSQLErrorResponsePacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.generic.PostgreSQLReadyForQueryPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLAuthenticationMD5PasswordPacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLAuthenticationOKPacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLComStartupPacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLParameterStatusPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLPasswordMessagePacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLRandomGenerator;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLSSLNegativePacket;
 import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -42,8 +47,14 @@ public final class PostgreSQLAuthenticationEngine implements AuthenticationEngin
     
     private static final int SSL_REQUEST_CODE = 80877103;
     
+    private static final String USER_NAME_KEYWORD = "user";
+    
     private static final String DATABASE_NAME_KEYWORD = "database";
     
+    private final AtomicBoolean startupMessageReceived = new AtomicBoolean(false);
+    
+    private volatile byte[] md5Salt;
+    
     @Override
     public void handshake(final ChannelHandlerContext context, final BackendConnection backendConnection) {
         int connectionId = ConnectionIdGenerator.getInstance().nextId();
@@ -58,23 +69,65 @@ public final class PostgreSQLAuthenticationEngine implements AuthenticationEngin
             return false;
         }
         payload.getByteBuf().resetReaderIndex();
-        PostgreSQLComStartupPacket comStartupPacket = new PostgreSQLComStartupPacket((PostgreSQLPacketPayload) payload);
-        String databaseName = comStartupPacket.getParametersMap().get(DATABASE_NAME_KEYWORD);
-        if (!Strings.isNullOrEmpty(databaseName) && !ProxySchemaContexts.getInstance().schemaExists(databaseName)) {
-            PostgreSQLErrorResponsePacket responsePacket = new PostgreSQLErrorResponsePacket();
-            responsePacket.addField(PostgreSQLErrorResponsePacket.FIELD_TYPE_SEVERITY, "FATAL");
-            responsePacket.addField(PostgreSQLErrorResponsePacket.FIELD_TYPE_CODE, "3D000");
-            responsePacket.addField(PostgreSQLErrorResponsePacket.FIELD_TYPE_MESSAGE, String.format("database \"%s\" does not exist", databaseName));
-            context.writeAndFlush(responsePacket);
+        if (!startupMessageReceived.get()) {
+            PostgreSQLComStartupPacket comStartupPacket = new PostgreSQLComStartupPacket((PostgreSQLPacketPayload) payload);
+            startupMessageReceived.set(true);
+            String databaseName = comStartupPacket.getParametersMap().get(DATABASE_NAME_KEYWORD);
+            if (!Strings.isNullOrEmpty(databaseName) && !ProxySchemaContexts.getInstance().schemaExists(databaseName)) {
+                PostgreSQLErrorResponsePacket responsePacket = createPostgreSQLErrorResponsePacket(PostgreSQLErrorCode.INVALID_CATALOG_NAME,
+                    String.format("database \"%s\" does not exist", databaseName));
+                context.writeAndFlush(responsePacket);
+                context.close();
+                return false;
+            }
+            backendConnection.setCurrentSchema(databaseName);
+            String userName = comStartupPacket.getParametersMap().get(USER_NAME_KEYWORD);
+            if (null == userName || userName.isEmpty()) {
+                PostgreSQLErrorResponsePacket responsePacket = createPostgreSQLErrorResponsePacket(PostgreSQLErrorCode.SQLSERVER_REJECTED_ESTABLISHMENT_OF_SQLCONNECTION,
+                    "user not set in StartupMessage");
+                context.writeAndFlush(responsePacket);
+                context.close();
+                return false;
+            }
+            backendConnection.setUserName(userName);
+            md5Salt = PostgreSQLRandomGenerator.getInstance().generateRandomBytes(4);
+            context.writeAndFlush(new PostgreSQLAuthenticationMD5PasswordPacket(md5Salt));
             return false;
+        } else {
+            char messageType = (char) ((PostgreSQLPacketPayload) payload).readInt1();
+            if ('p' != messageType) {
+                PostgreSQLErrorResponsePacket responsePacket = createPostgreSQLErrorResponsePacket(PostgreSQLErrorCode.SQLSERVER_REJECTED_ESTABLISHMENT_OF_SQLCONNECTION,
+                    "PasswordMessage is expected, message type 'p', but not '" + messageType + "'");
+                context.writeAndFlush(responsePacket);
+                context.close();
+                return false;
+            }
+            PostgreSQLPasswordMessagePacket passwordMessagePacket = new PostgreSQLPasswordMessagePacket((PostgreSQLPacketPayload) payload);
+            PostgreSQLLoginResult loginResult = PostgreSQLAuthenticationHandler.loginWithMd5Password(
+                backendConnection.getUserName(), backendConnection.getSchema().getName(), md5Salt, passwordMessagePacket);
+            if (PostgreSQLErrorCode.SUCCESSFUL_COMPLETION != loginResult.getErrorCode()) {
+                PostgreSQLErrorResponsePacket responsePacket = createPostgreSQLErrorResponsePacket(loginResult.getErrorCode(),
+                    loginResult.getErrorMessage());
+                context.writeAndFlush(responsePacket);
+                context.close();
+                return false;
+            } else {
+                // TODO implement PostgreSQLServerInfo like MySQLServerInfo
+                context.write(new PostgreSQLAuthenticationOKPacket(true));
+                context.write(new PostgreSQLParameterStatusPacket("server_version", "12.3"));
+                context.write(new PostgreSQLParameterStatusPacket("client_encoding", "UTF8"));
+                context.write(new PostgreSQLParameterStatusPacket("server_encoding", "UTF8"));
+                context.writeAndFlush(new PostgreSQLReadyForQueryPacket());
+                return true;
+            }
         }
-        backendConnection.setCurrentSchema(databaseName);
-        // TODO send a md5 authentication request message
-        context.write(new PostgreSQLAuthenticationOKPacket(true));
-        context.write(new PostgreSQLParameterStatusPacket("server_version", "12.3"));
-        context.write(new PostgreSQLParameterStatusPacket("client_encoding", "UTF8"));
-        context.write(new PostgreSQLParameterStatusPacket("server_encoding", "UTF8"));
-        context.writeAndFlush(new PostgreSQLReadyForQueryPacket());
-        return true;
+    }
+    
+    private PostgreSQLErrorResponsePacket createPostgreSQLErrorResponsePacket(final PostgreSQLErrorCode errorCode, final String errorMessage) {
+        PostgreSQLErrorResponsePacket result = new PostgreSQLErrorResponsePacket();
+        result.addField(PostgreSQLErrorResponsePacket.FIELD_TYPE_SEVERITY, "FATAL");
+        result.addField(PostgreSQLErrorResponsePacket.FIELD_TYPE_CODE, errorCode.getErrorCode());
+        result.addField(PostgreSQLErrorResponsePacket.FIELD_TYPE_MESSAGE, Strings.isNullOrEmpty(errorMessage) ? errorCode.getConditionName() : errorMessage);
+        return result;
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandler.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandler.java
new file mode 100644
index 0000000..f95e015
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandler.java
@@ -0,0 +1,76 @@
+/*
+ * 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.shardingsphere.proxy.frontend.postgresql.auth;
+
+import java.security.MessageDigest;
+import java.util.Map;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLErrorCode;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLPasswordMessagePacket;
+import org.apache.shardingsphere.infra.auth.ProxyUser;
+import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
+
+/**
+ * Authentication handler for PostgreSQL.
+ */
+public class PostgreSQLAuthenticationHandler {
+    
+    /**
+     * Login.
+     *
+     * @param userName              user name
+     * @param databaseName          database name
+     * @param md5Salt               md5 salt
+     * @param passwordMessagePacket password message packet
+     * @return PostgreSQLLoginResult
+     */
+    public static PostgreSQLLoginResult loginWithMd5Password(final String userName, final String databaseName, final byte[] md5Salt, final PostgreSQLPasswordMessagePacket passwordMessagePacket) {
+        ProxyUser proxyUser = null;
+        for (Map.Entry<String, ProxyUser> entry : ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().entrySet()) {
+            if (entry.getKey().equals(userName)) {
+                proxyUser = entry.getValue();
+                break;
+            }
+        }
+        if (null == proxyUser) {
+            return new PostgreSQLLoginResult(PostgreSQLErrorCode.INVALID_AUTHORIZATION_SPECIFICATION, "unknown userName: " + userName);
+        }
+        
+        String md5Digest = passwordMessagePacket.getMd5Digest();
+        String expectedMd5Digest = md5Encode(userName, proxyUser.getPassword(), md5Salt);
+        if (!expectedMd5Digest.equals(md5Digest)) {
+            return new PostgreSQLLoginResult(PostgreSQLErrorCode.INVALID_PASSWORD, "password authentication failed for user \"" + userName + "\"");
+        }
+        
+        if (!proxyUser.getAuthorizedSchemas().contains(databaseName)) {
+            return new PostgreSQLLoginResult(PostgreSQLErrorCode.PRIVILEGE_NOT_GRANTED, String.format("%s has no configured %s in authorizedSchemas", userName, databaseName));
+        }
+        
+        return new PostgreSQLLoginResult(PostgreSQLErrorCode.SUCCESSFUL_COMPLETION, null);
+    }
+    
+    private static String md5Encode(final String userName, final String password, final byte[] md5Salt) {
+        String passwordHash = new String(Hex.encodeHex(DigestUtils.md5(password + userName), true));
+        MessageDigest messageDigest = DigestUtils.getMd5Digest();
+        messageDigest.update(passwordHash.getBytes());
+        messageDigest.update(md5Salt);
+        return "md5" + new String(Hex.encodeHex(messageDigest.digest(), true));
+    }
+    
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLLoginResult.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLLoginResult.java
new file mode 100644
index 0000000..d5109b1
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLLoginResult.java
@@ -0,0 +1,37 @@
+/*
+ * 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.shardingsphere.proxy.frontend.postgresql.auth;
+
+import lombok.Getter;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLErrorCode;
+
+/**
+ * PostgreSQL login result.
+ */
+@Getter
+public class PostgreSQLLoginResult {
+    
+    private final PostgreSQLErrorCode errorCode;
+    
+    private final String errorMessage;
+    
+    public PostgreSQLLoginResult(final PostgreSQLErrorCode errorCode, final String errorMessage) {
+        this.errorCode = errorCode;
+        this.errorMessage = errorMessage;
+    }
+}