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 2021/03/16 01:48:46 UTC

[shardingsphere] branch master updated: Support object array insertion for PostgreSQL proxy (#9661)

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 7d3b42c  Support object array insertion for PostgreSQL proxy (#9661)
7d3b42c is described below

commit 7d3b42c75d2ed5e1e28f88bcc56b47c3a193dcc7
Author: sandynz <42...@users.noreply.github.com>
AuthorDate: Tue Mar 16 09:48:13 2021 +0800

    Support object array insertion for PostgreSQL proxy (#9661)
---
 .../protocol/PostgreSQLArrayParameterDecoder.java  | 205 +++++++++++++++++++++
 .../PostgreSQLBinaryProtocolValueFactory.java      |  37 ++++
 .../PostgreSQLBoolArrayBinaryProtocolValue.java    |  46 +++++
 .../PostgreSQLFloat4ArrayBinaryProtocolValue.java  |  46 +++++
 .../PostgreSQLFloat8ArrayBinaryProtocolValue.java  |  46 +++++
 .../PostgreSQLInt2ArrayBinaryProtocolValue.java    |  46 +++++
 .../PostgreSQLInt4ArrayBinaryProtocolValue.java    |  46 +++++
 .../PostgreSQLInt8ArrayBinaryProtocolValue.java    |  46 +++++
 .../PostgreSQLStringArrayBinaryProtocolValue.java  |  46 +++++
 .../PostgreSQLArrayParameterDecoderTest.java       | 118 ++++++++++++
 ...PostgreSQLBoolArrayBinaryProtocolValueTest.java |  61 ++++++
 ...stgreSQLFloat4ArrayBinaryProtocolValueTest.java |  61 ++++++
 ...stgreSQLFloat8ArrayBinaryProtocolValueTest.java |  61 ++++++
 ...PostgreSQLInt2ArrayBinaryProtocolValueTest.java |  61 ++++++
 ...PostgreSQLInt4ArrayBinaryProtocolValueTest.java |  61 ++++++
 ...PostgreSQLInt8ArrayBinaryProtocolValueTest.java |  61 ++++++
 ...stgreSQLStringArrayBinaryProtocolValueTest.java |  61 ++++++
 17 files changed, 1109 insertions(+)

diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLArrayParameterDecoder.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLArrayParameterDecoder.java
new file mode 100644
index 0000000..0439ea9
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLArrayParameterDecoder.java
@@ -0,0 +1,205 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+/**
+ * PostgreSQL array parameter decoder.
+ */
+public final class PostgreSQLArrayParameterDecoder {
+    
+    private static final String[] EMPTY_STRING_ARRAY = new String[0];
+    
+    /**
+     * Decode int2 array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public short[] decodeInt2Array(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            short[] result = new short[parameterElements.size()];
+            int index = 0;
+            for (String element : parameterElements) {
+                result[index++] = Short.parseShort(element);
+            }
+            return result;
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode int4 array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public int[] decodeInt4Array(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            int[] result = new int[parameterElements.size()];
+            int index = 0;
+            for (String element : parameterElements) {
+                result[index++] = Integer.parseInt(element);
+            }
+            return result;
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode int8 array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public long[] decodeInt8Array(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            long[] result = new long[parameterElements.size()];
+            int index = 0;
+            for (String element : parameterElements) {
+                result[index++] = Long.parseLong(element);
+            }
+            return result;
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode float4 array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public float[] decodeFloat4Array(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            float[] result = new float[parameterElements.size()];
+            int index = 0;
+            for (String element : parameterElements) {
+                result[index++] = Float.parseFloat(element);
+            }
+            return result;
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode float8 array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public double[] decodeFloat8Array(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            double[] result = new double[parameterElements.size()];
+            int index = 0;
+            for (String element : parameterElements) {
+                result[index++] = Double.parseDouble(element);
+            }
+            return result;
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode bool array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public boolean[] decodeBoolArray(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            boolean[] result = new boolean[parameterElements.size()];
+            int index = 0;
+            for (String element : parameterElements) {
+                result[index++] = Boolean.parseBoolean(element);
+            }
+            return result;
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode string array parameter.
+     *
+     * @param parameterBytes protocol parameter value
+     * @param isBinary whether parameter value is binary or not
+     * @return int array
+     */
+    public String[] decodeStringArray(final byte[] parameterBytes, final boolean isBinary) {
+        if (!isBinary) {
+            String parameterValue = new String(parameterBytes, StandardCharsets.UTF_8);
+            Collection<String> parameterElements = decodeText(parameterValue);
+            return parameterElements.toArray(EMPTY_STRING_ARRAY);
+        }
+        throw new UnsupportedOperationException("binary mode");
+    }
+    
+    /**
+     * Decode parameter in text mode.
+     *
+     * @param value protocol parameter value
+     * @return decoded parameter value elements
+     */
+    private Collection<String> decodeText(final String value) {
+        if (value.length() < 2) {
+            throw new IllegalArgumentException("value length less than 2");
+        }
+        if ('{' != value.charAt(0) || '}' != value.charAt(value.length() - 1)) {
+            throw new IllegalArgumentException("value not start with '{' or not end with '}'");
+        }
+        String[] elements = value.substring(1, value.length() - 1).split(",");
+        return Arrays.stream(elements).map(e -> {
+            if ("NULL".equals(e)) {
+                return null;
+            }
+            if ('"' == e.charAt(0) && '"' == e.charAt(e.length() - 1)) {
+                e = e.substring(1, e.length() - 1);
+            }
+            while (e.contains("\\\"")) {
+                e = e.replace("\\\"", "\"");
+            }
+            while (e.contains("\\\\")) {
+                e = e.replace("\\\\", "\\");
+            }
+            return e;
+        }).collect(Collectors.toCollection(ArrayList::new));
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBinaryProtocolValueFactory.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBinaryProtocolValueFactory.java
index b587e50..e97b41d 100644
--- a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBinaryProtocolValueFactory.java
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBinaryProtocolValueFactory.java
@@ -45,6 +45,13 @@ public final class PostgreSQLBinaryProtocolValueFactory {
         setNumericBinaryProtocolValue();
         setDateBinaryProtocolValue();
         setTimeBinaryProtocolValue();
+        setInt2ArrayBinaryProtocolValue();
+        setInt4ArrayBinaryProtocolValue();
+        setInt8ArrayBinaryProtocolValue();
+        setFloat4ArrayBinaryProtocolValue();
+        setFloat8ArrayBinaryProtocolValue();
+        setBoolArrayBinaryProtocolValue();
+        setStringArrayBinaryProtocolValue();
     }
     
     private static void setUnspecifiedBinaryProtocolValue() {
@@ -97,6 +104,36 @@ public final class PostgreSQLBinaryProtocolValueFactory {
         BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_TIMESTAMP, binaryProtocolValue);
     }
     
+    private static void setInt2ArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_INT2_ARRAY, new PostgreSQLInt2ArrayBinaryProtocolValue());
+    }
+    
+    private static void setInt4ArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_INT4_ARRAY, new PostgreSQLInt4ArrayBinaryProtocolValue());
+    }
+    
+    private static void setInt8ArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_INT8_ARRAY, new PostgreSQLInt8ArrayBinaryProtocolValue());
+    }
+    
+    private static void setFloat4ArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_FLOAT4_ARRAY, new PostgreSQLFloat4ArrayBinaryProtocolValue());
+    }
+    
+    private static void setFloat8ArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_FLOAT8_ARRAY, new PostgreSQLFloat8ArrayBinaryProtocolValue());
+    }
+    
+    private static void setBoolArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_BOOL_ARRAY, new PostgreSQLBoolArrayBinaryProtocolValue());
+    }
+    
+    private static void setStringArrayBinaryProtocolValue() {
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_VARCHAR_ARRAY, new PostgreSQLStringArrayBinaryProtocolValue());
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_CHAR_ARRAY, new PostgreSQLStringArrayBinaryProtocolValue());
+        BINARY_PROTOCOL_VALUES.put(PostgreSQLBinaryColumnType.POSTGRESQL_TYPE_TEXT_ARRAY, new PostgreSQLStringArrayBinaryProtocolValue());
+    }
+    
     /**
      * Get binary protocol value.
      *
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBoolArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBoolArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..25f53d2
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBoolArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for boolean array for PostgreSQL.
+ */
+public final class PostgreSQLBoolArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeBoolArray(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat4ArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat4ArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..c4af6a1
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat4ArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for float4 array for PostgreSQL.
+ */
+public final class PostgreSQLFloat4ArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeFloat4Array(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat8ArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat8ArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..5adb06b
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat8ArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for float8 array for PostgreSQL.
+ */
+public final class PostgreSQLFloat8ArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeFloat8Array(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt2ArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt2ArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..4494601
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt2ArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for int2 array for PostgreSQL.
+ */
+public final class PostgreSQLInt2ArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeInt2Array(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt4ArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt4ArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..6b02ae4
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt4ArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for int4 array for PostgreSQL.
+ */
+public final class PostgreSQLInt4ArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeInt4Array(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt8ArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt8ArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..56ced5a
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt8ArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for int8 array for PostgreSQL.
+ */
+public final class PostgreSQLInt8ArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeInt8Array(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLStringArrayBinaryProtocolValue.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLStringArrayBinaryProtocolValue.java
new file mode 100644
index 0000000..ed953ce
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLStringArrayBinaryProtocolValue.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.command.query.binary.bind.protocol;
+
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Binary protocol value for string array for PostgreSQL.
+ */
+public final class PostgreSQLStringArrayBinaryProtocolValue implements PostgreSQLBinaryProtocolValue {
+    
+    private static final PostgreSQLArrayParameterDecoder ARRAY_PARAMETER_DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    @Override
+    public int getColumnLength(final Object value) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Object read(final PostgreSQLPacketPayload payload) {
+        payload.getByteBuf().readerIndex(payload.getByteBuf().readerIndex() - 4);
+        byte[] bytes = new byte[payload.readInt4()];
+        payload.getByteBuf().readBytes(bytes);
+        return ARRAY_PARAMETER_DECODER.decodeStringArray(bytes, '{' != bytes[0]);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload, final Object value) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLArrayParameterDecoderTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLArrayParameterDecoderTest.java
new file mode 100644
index 0000000..58718cf
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLArrayParameterDecoderTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLArrayParameterDecoderTest {
+    
+    private static final PostgreSQLArrayParameterDecoder DECODER = new PostgreSQLArrayParameterDecoder();
+    
+    private static final String INT_ARRAY_STR = "{\"11\",\"12\"}";
+    
+    private static final String FLOAT_ARRAY_STR = "{\"11.1\",\"12.1\"}";
+    
+    @Test
+    public void assertParseInt2ArrayNormalTextMode() {
+        short[] result = DECODER.decodeInt2Array(INT_ARRAY_STR.getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(result[0], is((short) 11));
+        assertThat(result[1], is((short) 12));
+    }
+    
+    @Test
+    public void assertParseInt4ArrayNormalTextMode() {
+        int[] result = DECODER.decodeInt4Array(INT_ARRAY_STR.getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(result[0], is(11));
+        assertThat(result[1], is(12));
+    }
+    
+    @Test
+    public void assertParseInt8ArrayNormalTextMode() {
+        long[] result = DECODER.decodeInt8Array(INT_ARRAY_STR.getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(result[0], is(11L));
+        assertThat(result[1], is(12L));
+    }
+    
+    @Test
+    public void assertParseFloat4ArrayNormalTextMode() {
+        float[] result = DECODER.decodeFloat4Array(FLOAT_ARRAY_STR.getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(Float.compare(result[0], 11.1F), is(0));
+        assertThat(Float.compare(result[1], 12.1F), is(0));
+    }
+    
+    @Test
+    public void assertParseFloat8ArrayNormalTextMode() {
+        double[] result = DECODER.decodeFloat8Array(FLOAT_ARRAY_STR.getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(Double.compare(result[0], 11.1D), is(0));
+        assertThat(Double.compare(result[1], 12.1D), is(0));
+    }
+    
+    @Test
+    public void assertParseBoolArrayNormalTextMode() {
+        boolean[] result = DECODER.decodeBoolArray("{\"true\",\"false\"}".getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(result[0], is(true));
+        assertThat(result[1], is(false));
+    }
+    
+    @Test
+    public void assertParseStringArrayNormalTextMode() {
+        String[] result = DECODER.decodeStringArray("{\"a\",\"b\"}".getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(2));
+        assertThat(result[0], is("a"));
+        assertThat(result[1], is("b"));
+    }
+    
+    @Test
+    public void assertParseStringArrayWithEscapeTextMode() {
+        String[] result = DECODER.decodeStringArray("{\"\\\"a\",\"\\\\b\",\"c\"}".getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(3));
+        assertThat(result[0], is("\"a"));
+        assertThat(result[1], is("\\b"));
+        assertThat(result[2], is("c"));
+    }
+    
+    @Test
+    public void assertParseStringArrayWithNullTextMode() {
+        String[] result = DECODER.decodeStringArray("{\"a\",\"b\",NULL}".getBytes(), false);
+        assertNotNull(result);
+        assertThat(result.length, is(3));
+        assertThat(result[0], is("a"));
+        assertThat(result[1], is("b"));
+        assertNull(result[2]);
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBoolArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBoolArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..cbb5c75
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLBoolArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLBoolArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLBoolArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"true\",\"false\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new boolean[] {true, false}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat4ArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat4ArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..7328d59
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat4ArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLFloat4ArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLFloat4ArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"11.1\",\"12.1\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new float[] {11.1F, 12.1F}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat8ArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat8ArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..63d2deb
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLFloat8ArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLFloat8ArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLFloat8ArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"11.1\",\"12.1\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new double[] {11.1D, 12.1D}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt2ArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt2ArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..94e897c
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt2ArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLInt2ArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLInt2ArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"11\",\"12\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new short[] {11, 12}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt4ArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt4ArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..f4cb93b
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt4ArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLInt4ArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLInt4ArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"11\",\"12\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new int[] {11, 12}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt8ArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt8ArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..fae8b42
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLInt8ArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLInt8ArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLInt8ArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"11\",\"12\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new long[] {11, 12}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLStringArrayBinaryProtocolValueTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLStringArrayBinaryProtocolValueTest.java
new file mode 100644
index 0000000..8028a96
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/binary/bind/protocol/PostgreSQLStringArrayBinaryProtocolValueTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.command.query.binary.bind.protocol;
+
+import io.netty.buffer.ByteBuf;
+import java.nio.charset.StandardCharsets;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.ByteBufTestUtils;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public final class PostgreSQLStringArrayBinaryProtocolValueTest {
+    
+    private PostgreSQLBinaryProtocolValue newInstance() {
+        return new PostgreSQLStringArrayBinaryProtocolValue();
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertGetColumnLength() {
+        newInstance().getColumnLength("val");
+    }
+    
+    @Test
+    public void assertRead() {
+        String parameterValue = "{\"a\",\"b\"}";
+        int expectedLength = 4 + parameterValue.length();
+        ByteBuf byteBuf = ByteBufTestUtils.createByteBuf(expectedLength);
+        byteBuf.writeInt(parameterValue.length());
+        byteBuf.writeCharSequence(parameterValue, StandardCharsets.ISO_8859_1);
+        byteBuf.readInt();
+        PostgreSQLPacketPayload payload = new PostgreSQLPacketPayload(byteBuf);
+        Object result = newInstance().read(payload);
+        assertNotNull(result);
+        assertThat(result, is(new String[] {"a", "b"}));
+        assertThat(byteBuf.readerIndex(), is(expectedLength));
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertWrite() {
+        newInstance().write(new PostgreSQLPacketPayload(null), "val");
+    }
+    
+}