You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2021/11/12 01:25:51 UTC

[shardingsphere] branch master updated: Add shadow value validator. (#13559)

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

panjuan 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 d390ec7  Add shadow value validator. (#13559)
d390ec7 is described below

commit d390ec75fb2bc553fa52ddee72b8196d825c143b
Author: gin <ja...@163.com>
AuthorDate: Fri Nov 12 09:25:13 2021 +0800

    Add shadow value validator. (#13559)
---
 .../column/AbstractColumnMatchShadowAlgorithm.java | 97 ++++++++++++++++++++++
 .../column/ColumnRegexMatchShadowAlgorithm.java    | 60 ++-----------
 .../column/ColumnValueMatchShadowAlgorithm.java    | 60 ++-----------
 .../shadow/validator/ShadowValueValidator.java     | 33 ++++++++
 .../validator/column/ShadowDateValueValidator.java | 36 ++++++++
 .../validator/column/ShadowEnumValueValidator.java | 34 ++++++++
 .../column/AbstractColumnShadowAlgorithmTest.java  | 12 +++
 .../ColumnRegexMatchShadowAlgorithmTest.java       |  6 ++
 .../ColumnValueMatchShadowAlgorithmTest.java       |  6 ++
 9 files changed, 236 insertions(+), 108 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java
new file mode 100644
index 0000000..24d5944
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnMatchShadowAlgorithm.java
@@ -0,0 +1,97 @@
+/*
+ * 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.shadow.algorithm.shadow.column;
+
+import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.shadow.algorithm.shadow.validator.ShadowValueValidator;
+import org.apache.shardingsphere.shadow.algorithm.shadow.validator.column.ShadowDateValueValidator;
+import org.apache.shardingsphere.shadow.algorithm.shadow.validator.column.ShadowEnumValueValidator;
+import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
+import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
+import org.apache.shardingsphere.shadow.api.shadow.column.PreciseColumnShadowValue;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+import java.util.Properties;
+
+/**
+ * Abstract column match shadow algorithm.
+ */
+public abstract class AbstractColumnMatchShadowAlgorithm implements ColumnShadowAlgorithm<Comparable<?>> {
+    
+    private static final String COLUMN_PROPS_KEY = "column";
+    
+    private static final String OPERATION_PROPS_KEY = "operation";
+    
+    private static final Collection<ShadowValueValidator> SHADOW_VALUE_VALIDATORS = new LinkedList<>();
+    
+    @Getter
+    @Setter
+    private Properties props = new Properties();
+    
+    private ShadowOperationType shadowOperationType;
+    
+    @Override
+    public boolean isShadow(final Collection<String> relatedShadowTables, final PreciseColumnShadowValue<Comparable<?>> shadowValue) {
+        String table = shadowValue.getLogicTableName();
+        String column = shadowValue.getColumnName();
+        Comparable<?> value = shadowValue.getValue();
+        SHADOW_VALUE_VALIDATORS.forEach(each -> each.preValidate(table, column, value));
+        return shadowOperationType == shadowValue.getShadowOperationType() && relatedShadowTables.contains(table) && String.valueOf(props.get(COLUMN_PROPS_KEY)).equals(column) && isMatchValue(value);
+    }
+    
+    @Override
+    public void init() {
+        checkColumn();
+        checkOperation();
+        checkProps();
+        initShadowValueValidator();
+    }
+    
+    private void checkColumn() {
+        Preconditions.checkNotNull(props.get(COLUMN_PROPS_KEY), "Column shadow algorithm column cannot be null.");
+    }
+    
+    private void checkOperation() {
+        String operationType = String.valueOf(props.get(OPERATION_PROPS_KEY));
+        Preconditions.checkNotNull(operationType, "Column shadow algorithm operation cannot be null.");
+        Optional<ShadowOperationType> shadowOperationType = ShadowOperationType.contains(operationType);
+        Preconditions.checkState(shadowOperationType.isPresent(), "Column shadow algorithm operation must be one of [select, insert, update, delete].");
+        this.shadowOperationType = shadowOperationType.get();
+    }
+    
+    private void initShadowValueValidator() {
+        SHADOW_VALUE_VALIDATORS.add(new ShadowDateValueValidator());
+        SHADOW_VALUE_VALIDATORS.add(new ShadowEnumValueValidator());
+    }
+    
+    /**
+     * Check props.
+     */
+    protected abstract void checkProps();
+    
+    /**
+     * Is matching of value.
+     * @param value value
+     * @return is matching or not
+     */
+    protected abstract boolean isMatchValue(Comparable<?> value);
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java
index e3ce367..63976ea 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithm.java
@@ -18,70 +18,22 @@
 package org.apache.shardingsphere.shadow.algorithm.shadow.column;
 
 import com.google.common.base.Preconditions;
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
-import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
-import org.apache.shardingsphere.shadow.api.shadow.column.PreciseColumnShadowValue;
-
-import java.util.Collection;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 
 /**
  * Column regex match shadow algorithm.
  */
-@Getter
-@Setter
-public final class ColumnRegexMatchShadowAlgorithm implements ColumnShadowAlgorithm<Comparable<?>> {
-    
-    private static final String COLUMN = "column";
-    
-    private static final String OPERATION = "operation";
-    
-    private static final String REGEX = "regex";
+public final class ColumnRegexMatchShadowAlgorithm extends AbstractColumnMatchShadowAlgorithm {
     
-    private Properties props = new Properties();
-    
-    private ShadowOperationType shadowOperationType;
+    private static final String REGEX_PROPS_KEY = "regex";
     
     @Override
-    public void init() {
-        checkProps();
-    }
-    
-    private void checkProps() {
-        checkOperation();
-        checkColumn();
-        checkRegex();
-    }
-    
-    private void checkRegex() {
-        String regex = props.getProperty(REGEX);
-        Preconditions.checkNotNull(regex, "Column regex match shadow algorithm regex cannot be null.");
-    }
-    
-    private void checkColumn() {
-        String column = props.getProperty(COLUMN);
-        Preconditions.checkNotNull(column, "Column regex match shadow algorithm column cannot be null.");
-    }
-    
-    private void checkOperation() {
-        String operationType = props.getProperty(OPERATION);
-        Preconditions.checkNotNull(operationType, "Column regex match shadow algorithm operation cannot be null.");
-        Optional<ShadowOperationType> shadowOperationType = ShadowOperationType.contains(operationType);
-        Preconditions.checkState(shadowOperationType.isPresent(), "Column regex match shadow algorithm operation must be one of select insert update delete.");
-        shadowOperationType.ifPresent(type -> this.shadowOperationType = type);
+    protected void checkProps() {
+        Preconditions.checkNotNull(getProps().get(REGEX_PROPS_KEY), "Column regex match shadow algorithm regex cannot be null.");
     }
     
     @Override
-    public boolean isShadow(final Collection<String> shadowTableNames, final PreciseColumnShadowValue<Comparable<?>> shadowValue) {
-        boolean containTable = shadowTableNames.contains(shadowValue.getLogicTableName());
-        boolean isSameOperation = shadowOperationType == shadowValue.getShadowOperationType();
-        boolean isSameColumnName = Objects.equals(props.get(COLUMN), shadowValue.getColumnName());
-        boolean isRegexMatch = String.valueOf(shadowValue.getValue()).matches(props.get(REGEX).toString());
-        return containTable && isSameOperation && isSameColumnName && isRegexMatch;
+    protected boolean isMatchValue(final Comparable<?> value) {
+        return String.valueOf(value).matches(String.valueOf(getProps().get(REGEX_PROPS_KEY)));
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithm.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithm.java
index e934b6d..7b842cf 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithm.java
@@ -18,70 +18,22 @@
 package org.apache.shardingsphere.shadow.algorithm.shadow.column;
 
 import com.google.common.base.Preconditions;
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
-import org.apache.shardingsphere.shadow.api.shadow.column.ColumnShadowAlgorithm;
-import org.apache.shardingsphere.shadow.api.shadow.column.PreciseColumnShadowValue;
-
-import java.util.Collection;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 
 /**
  * Column value match shadow algorithm.
  */
-@Getter
-@Setter
-public final class ColumnValueMatchShadowAlgorithm implements ColumnShadowAlgorithm<Comparable<?>> {
-    
-    private static final String COLUMN = "column";
-    
-    private static final String OPERATION = "operation";
-    
-    private static final String VALUE = "value";
+public final class ColumnValueMatchShadowAlgorithm extends AbstractColumnMatchShadowAlgorithm {
     
-    private Properties props = new Properties();
-    
-    private ShadowOperationType shadowOperationType;
+    private static final String VALUE_PROPS_KEY = "value";
     
     @Override
-    public void init() {
-        checkProps();
-    }
-    
-    private void checkProps() {
-        checkOperation();
-        checkColumn();
-        checkValue();
-    }
-    
-    private void checkValue() {
-        Object value = props.get(VALUE);
-        Preconditions.checkNotNull(value, "Column value match shadow algorithm value cannot be null.");
-    }
-    
-    private void checkColumn() {
-        String column = props.getProperty(COLUMN);
-        Preconditions.checkNotNull(column, "Column value match shadow algorithm column cannot be null.");
-    }
-    
-    private void checkOperation() {
-        String operationType = props.getProperty(OPERATION);
-        Preconditions.checkNotNull(operationType, "Column value match shadow algorithm operation cannot be null.");
-        Optional<ShadowOperationType> shadowOperationType = ShadowOperationType.contains(operationType);
-        Preconditions.checkState(shadowOperationType.isPresent(), "Column value match shadow algorithm operation must be one of select insert update delete.");
-        shadowOperationType.ifPresent(type -> this.shadowOperationType = type);
+    protected void checkProps() {
+        Preconditions.checkNotNull(getProps().get(VALUE_PROPS_KEY), "Column value match shadow algorithm value cannot be null.");
     }
     
     @Override
-    public boolean isShadow(final Collection<String> shadowTableNames, final PreciseColumnShadowValue<Comparable<?>> shadowValue) {
-        boolean containTable = shadowTableNames.contains(shadowValue.getLogicTableName());
-        boolean isSameOperation = shadowOperationType == shadowValue.getShadowOperationType();
-        boolean isSameColumnName = Objects.equals(props.get(COLUMN), shadowValue.getColumnName());
-        boolean isSameColumnValue = props.get(VALUE).toString().equals(String.valueOf(shadowValue.getValue()));
-        return containTable && isSameOperation && isSameColumnName && isSameColumnValue;
+    protected boolean isMatchValue(final Comparable<?> value) {
+        return String.valueOf(getProps().get(VALUE_PROPS_KEY)).equals(String.valueOf(value));
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/ShadowValueValidator.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/ShadowValueValidator.java
new file mode 100644
index 0000000..6688e8b
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/ShadowValueValidator.java
@@ -0,0 +1,33 @@
+/*
+ * 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.shadow.algorithm.shadow.validator;
+
+/**
+ * Shadow value validator.
+ */
+public interface ShadowValueValidator {
+    
+    /**
+     * Validate shadow value.
+     *
+     * @param table table name
+     * @param column column name
+     * @param shadowValue shadow value
+     */
+    void preValidate(String table, String column, Comparable<?> shadowValue);
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/column/ShadowDateValueValidator.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/column/ShadowDateValueValidator.java
new file mode 100644
index 0000000..7a2a1e8
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/column/ShadowDateValueValidator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.shadow.algorithm.shadow.validator.column;
+
+import org.apache.shardingsphere.shadow.algorithm.shadow.ShadowAlgorithmException;
+import org.apache.shardingsphere.shadow.algorithm.shadow.validator.ShadowValueValidator;
+
+import java.util.Date;
+
+/**
+ * Shadow value validator of date type.
+ */
+public final class ShadowDateValueValidator implements ShadowValueValidator {
+    
+    @Override
+    public void preValidate(final String table, final String column, final Comparable<?> shadowValue) {
+        if (shadowValue instanceof Date) {
+            throw new ShadowAlgorithmException("Shadow column `%s` data of shadow table `%s` matching does not support type: `%s`.", column, table, Date.class);
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/column/ShadowEnumValueValidator.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/column/ShadowEnumValueValidator.java
new file mode 100644
index 0000000..6241c06
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/validator/column/ShadowEnumValueValidator.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.shadow.algorithm.shadow.validator.column;
+
+import org.apache.shardingsphere.shadow.algorithm.shadow.ShadowAlgorithmException;
+import org.apache.shardingsphere.shadow.algorithm.shadow.validator.ShadowValueValidator;
+
+/**
+ * Shadow value validator of enum.
+ */
+public final class ShadowEnumValueValidator implements ShadowValueValidator {
+    
+    @Override
+    public void preValidate(final String table, final String column, final Comparable<?> shadowValue) {
+        if (shadowValue instanceof Enum) {
+            throw new ShadowAlgorithmException("Shadow column `%s` data of shadow table `%s` matching does not support type: `%s`.", column, table, Enum.class);
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnShadowAlgorithmTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnShadowAlgorithmTest.java
index c1ab938..803f5ad 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnShadowAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/AbstractColumnShadowAlgorithmTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.shadow.api.shadow.column.PreciseColumnShadowVal
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.Collection;
+import java.util.Date;
 import java.util.LinkedList;
 
 public abstract class AbstractColumnShadowAlgorithmTest {
@@ -31,6 +32,10 @@ public abstract class AbstractColumnShadowAlgorithmTest {
     
     protected static final String SHADOW_COLUMN = "shadow";
     
+    enum ShadowEnum {
+        SHADOW_VALUE
+    }
+    
     protected Collection<PreciseColumnShadowValue<Comparable<?>>> createPreciseColumnShadowValuesTrueCase() {
         Collection<PreciseColumnShadowValue<Comparable<?>>> result = new LinkedList<>();
         result.add(new PreciseColumnShadowValue<>(SHADOW_TABLE, ShadowOperationType.INSERT, SHADOW_COLUMN, 1));
@@ -65,6 +70,13 @@ public abstract class AbstractColumnShadowAlgorithmTest {
         return result;
     }
     
+    protected Collection<PreciseColumnShadowValue<Comparable<?>>> createPreciseColumnShadowValuesExceptionCase() {
+        Collection<PreciseColumnShadowValue<Comparable<?>>> result = new LinkedList<>();
+        result.add(new PreciseColumnShadowValue<>(SHADOW_TABLE, ShadowOperationType.INSERT, SHADOW_COLUMN, new Date()));
+        result.add(new PreciseColumnShadowValue<>(SHADOW_TABLE, ShadowOperationType.INSERT, SHADOW_COLUMN, ShadowEnum.SHADOW_VALUE));
+        return result;
+    }
+    
     protected Collection<String> createTableNames() {
         Collection<String> shadowTableNames = new LinkedList<>();
         shadowTableNames.add(SHADOW_TABLE);
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java
index 62c2338..3e53f3a 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnRegexMatchShadowAlgorithmTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.shadow.algorithm.shadow.column;
 
+import org.apache.shardingsphere.shadow.algorithm.shadow.ShadowAlgorithmException;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -54,6 +55,11 @@ public final class ColumnRegexMatchShadowAlgorithmTest extends AbstractColumnSha
         createPreciseColumnShadowValuesTrueCase().forEach(each -> assertThat(shadowAlgorithm.isShadow(createTableNames(), each), is(true)));
     }
     
+    @Test(expected = ShadowAlgorithmException.class)
+    public void assertExceptionCase() {
+        createPreciseColumnShadowValuesExceptionCase().forEach(each -> assertThat(shadowAlgorithm.isShadow(createTableNames(), each), is(false)));
+    }
+    
     @Test
     public void assertGetType() {
         assertThat(shadowAlgorithm.getType(), is("REGEX_MATCH"));
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithmTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithmTest.java
index 2e19a52..6b18ca3 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithmTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/column/ColumnValueMatchShadowAlgorithmTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.shadow.algorithm.shadow.column;
 
+import org.apache.shardingsphere.shadow.algorithm.shadow.ShadowAlgorithmException;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -54,6 +55,11 @@ public final class ColumnValueMatchShadowAlgorithmTest extends AbstractColumnSha
         createPreciseColumnShadowValuesFalseCase().forEach(each -> assertThat(shadowAlgorithm.isShadow(createTableNames(), each), is(false)));
     }
     
+    @Test(expected = ShadowAlgorithmException.class)
+    public void assertExceptionCase() {
+        createPreciseColumnShadowValuesExceptionCase().forEach(each -> assertThat(shadowAlgorithm.isShadow(createTableNames(), each), is(false)));
+    }
+    
     @Test
     public void assertGetType() {
         assertThat(shadowAlgorithm.getType(), is("VALUE_MATCH"));