You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2020/06/04 10:56:03 UTC

[hbase] branch master updated: HBASE-24456 : Create ImmutableScan and use it for CustomizedScanInfoBuilder (#1818)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 61a7468  HBASE-24456 : Create ImmutableScan and use it for CustomizedScanInfoBuilder (#1818)
61a7468 is described below

commit 61a7468df7412874d2fb492a8bab5d0d72ac6fcd
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Thu Jun 4 16:25:47 2020 +0530

    HBASE-24456 : Create ImmutableScan and use it for CustomizedScanInfoBuilder (#1818)
    
    Signed-off-by: Anoop Sam John <an...@apache.org>
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../apache/hadoop/hbase/client/ImmutableScan.java  | 542 +++++++++++++++++++++
 .../hadoop/hbase/client/TestImmutableScan.java     | 392 +++++++++++++++
 .../regionserver/CustomizedScanInfoBuilder.java    |  16 +-
 3 files changed, 939 insertions(+), 11 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableScan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableScan.java
new file mode 100644
index 0000000..535b926
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableScan.java
@@ -0,0 +1,542 @@
+/*
+ *
+ * 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.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableSet;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.visibility.Authorizations;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Immutable version of Scan
+ */
+@InterfaceAudience.Private
+public final class ImmutableScan extends Scan {
+
+  private final Scan delegateScan;
+
+  /**
+   * Create Immutable instance of Scan from given Scan object
+   *
+   * @param scan Copy all values from Scan
+   */
+  public ImmutableScan(Scan scan) {
+    this.delegateScan = scan;
+  }
+
+  @Override
+  public Scan addFamily(byte[] family) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to addFamily");
+  }
+
+  @Override
+  public Scan addColumn(byte[] family, byte[] qualifier) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to addColumn");
+  }
+
+  @Override
+  public Scan setTimeRange(long minStamp, long maxStamp) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setTimeRange");
+  }
+
+  @Deprecated
+  @Override
+  public Scan setTimeStamp(long timestamp) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setTimeStamp");
+  }
+
+  @Override
+  public Scan setTimestamp(long timestamp) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setTimestamp");
+  }
+
+  @Override
+  public Scan setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setColumnFamilyTimeRange");
+  }
+
+  @Override
+  public Scan withStartRow(byte[] startRow) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to withStartRow");
+  }
+
+  @Override
+  public Scan withStartRow(byte[] startRow, boolean inclusive) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to withStartRow");
+  }
+
+  @Override
+  public Scan withStopRow(byte[] stopRow) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to withStopRow");
+  }
+
+  @Override
+  public Scan withStopRow(byte[] stopRow, boolean inclusive) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to withStopRow");
+  }
+
+  @Override
+  public Scan setRowPrefixFilter(byte[] rowPrefix) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setRowPrefixFilter");
+  }
+
+  @Override
+  public Scan readAllVersions() {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to readAllVersions");
+  }
+
+  @Override
+  public Scan readVersions(int versions) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to readVersions");
+  }
+
+  @Override
+  public Scan setBatch(int batch) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setBatch");
+  }
+
+  @Override
+  public Scan setMaxResultsPerColumnFamily(int limit) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setMaxResultsPerColumnFamily");
+  }
+
+  @Override
+  public Scan setRowOffsetPerColumnFamily(int offset) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setRowOffsetPerColumnFamily");
+  }
+
+  @Override
+  public Scan setCaching(int caching) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setCaching");
+  }
+
+  @Override
+  public Scan setMaxResultSize(long maxResultSize) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setMaxResultSize");
+  }
+
+  @Override
+  public Scan setFilter(Filter filter) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setFilter");
+  }
+
+  @Override
+  public Scan setFamilyMap(Map<byte[], NavigableSet<byte[]>> familyMap) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setFamilyMap");
+  }
+
+  @Override
+  public Scan setCacheBlocks(boolean cacheBlocks) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setCacheBlocks");
+  }
+
+  @Override
+  public Scan setReversed(boolean reversed) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setReversed");
+  }
+
+  @Override
+  public Scan setAllowPartialResults(final boolean allowPartialResults) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setAllowPartialResults");
+  }
+
+  @Override
+  public Scan setLoadColumnFamiliesOnDemand(boolean value) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setLoadColumnFamiliesOnDemand");
+  }
+
+  @Override
+  public Scan setRaw(boolean raw) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setRaw");
+  }
+
+  @Override
+  @Deprecated
+  public Scan setSmall(boolean small) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setSmall");
+  }
+
+  @Override
+  public Scan setAttribute(String name, byte[] value) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setAttribute");
+  }
+
+  @Override
+  public Scan setId(String id) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setId");
+  }
+
+  @Override
+  public Scan setAuthorizations(Authorizations authorizations) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setAuthorizations");
+  }
+
+  @Override
+  public Scan setACL(Map<String, Permission> perms) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setACL");
+  }
+
+  @Override
+  public Scan setACL(String user, Permission perms) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setACL");
+  }
+
+  @Override
+  public Scan setConsistency(Consistency consistency) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setConsistency");
+  }
+
+  @Override
+  public Scan setReplicaId(int id) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setReplicaId");
+  }
+
+  @Override
+  public Scan setIsolationLevel(IsolationLevel level) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setIsolationLevel");
+  }
+
+  @Override
+  public Scan setPriority(int priority) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setPriority");
+  }
+
+  @Override
+  public Scan setScanMetricsEnabled(final boolean enabled) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setScanMetricsEnabled");
+  }
+
+  @Override
+  @Deprecated
+  public Scan setAsyncPrefetch(boolean asyncPrefetch) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setAsyncPrefetch");
+  }
+
+  @Override
+  public Scan setLimit(int limit) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setLimit");
+  }
+
+  @Override
+  public Scan setOneRowLimit() {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setOneRowLimit");
+  }
+
+  @Override
+  public Scan setReadType(ReadType readType) {
+    throw new UnsupportedOperationException("ImmutableScan does not allow access to setReadType");
+  }
+
+  @Override
+  Scan setMvccReadPoint(long mvccReadPoint) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setMvccReadPoint");
+  }
+
+  @Override
+  Scan resetMvccReadPoint() {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to resetMvccReadPoint");
+  }
+
+  @Override
+  public Scan setNeedCursorResult(boolean needCursorResult) {
+    throw new UnsupportedOperationException(
+      "ImmutableScan does not allow access to setNeedCursorResult");
+  }
+
+  @Override
+  public long getMaxResultSize() {
+    return this.delegateScan.getMaxResultSize();
+  }
+
+  @Override
+  public Map<byte[], NavigableSet<byte[]>> getFamilyMap() {
+    return Collections.unmodifiableMap(this.delegateScan.getFamilyMap());
+  }
+
+  @Override
+  public int numFamilies() {
+    return this.delegateScan.numFamilies();
+  }
+
+  @Override
+  public boolean hasFamilies() {
+    return this.delegateScan.hasFamilies();
+  }
+
+  @Override
+  public byte[][] getFamilies() {
+    final byte[][] families = this.delegateScan.getFamilies();
+    byte[][] cloneFamilies = new byte[families.length][];
+    for (int i = 0; i < families.length; i++) {
+      cloneFamilies[i] = Bytes.copy(families[i]);
+    }
+    return cloneFamilies;
+  }
+
+  @Override
+  public byte[] getStartRow() {
+    final byte[] startRow = this.delegateScan.getStartRow();
+    return Bytes.copy(startRow);
+  }
+
+  @Override
+  public boolean includeStartRow() {
+    return this.delegateScan.includeStartRow();
+  }
+
+  @Override
+  public byte[] getStopRow() {
+    final byte[] stopRow = this.delegateScan.getStopRow();
+    return Bytes.copy(stopRow);
+  }
+
+  @Override
+  public boolean includeStopRow() {
+    return this.delegateScan.includeStopRow();
+  }
+
+  @Override
+  public int getMaxVersions() {
+    return this.delegateScan.getMaxVersions();
+  }
+
+  @Override
+  public int getBatch() {
+    return this.delegateScan.getBatch();
+  }
+
+  @Override
+  public int getMaxResultsPerColumnFamily() {
+    return this.delegateScan.getMaxResultsPerColumnFamily();
+  }
+
+  @Override
+  public int getRowOffsetPerColumnFamily() {
+    return this.delegateScan.getRowOffsetPerColumnFamily();
+  }
+
+  @Override
+  public int getCaching() {
+    return this.delegateScan.getCaching();
+  }
+
+  @Override
+  public TimeRange getTimeRange() {
+    return this.delegateScan.getTimeRange();
+  }
+
+  @Override
+  public Filter getFilter() {
+    return this.delegateScan.getFilter();
+  }
+
+  @Override
+  public boolean hasFilter() {
+    return this.delegateScan.hasFilter();
+  }
+
+  @Override
+  public boolean getCacheBlocks() {
+    return this.delegateScan.getCacheBlocks();
+  }
+
+  @Override
+  public boolean isReversed() {
+    return this.delegateScan.isReversed();
+  }
+
+  @Override
+  public boolean getAllowPartialResults() {
+    return this.delegateScan.getAllowPartialResults();
+  }
+
+  @Override
+  public byte[] getACL() {
+    final byte[] acl = this.delegateScan.getACL();
+    return Bytes.copy(acl);
+  }
+
+  @Override
+  public Map<String, Object> getFingerprint() {
+    return Collections.unmodifiableMap(this.delegateScan.getFingerprint());
+  }
+
+  @Override
+  public Map<String, Object> toMap(int maxCols) {
+    return Collections.unmodifiableMap(this.delegateScan.toMap(maxCols));
+  }
+
+  @Override
+  public boolean isRaw() {
+    return this.delegateScan.isRaw();
+  }
+
+  @Override
+  @Deprecated
+  public boolean isSmall() {
+    return this.delegateScan.isSmall();
+  }
+
+  @Override
+  public boolean isScanMetricsEnabled() {
+    return this.delegateScan.isScanMetricsEnabled();
+  }
+
+  @Override
+  public Boolean isAsyncPrefetch() {
+    return this.delegateScan.isAsyncPrefetch();
+  }
+
+  @Override
+  public int getLimit() {
+    return this.delegateScan.getLimit();
+  }
+
+  @Override
+  public ReadType getReadType() {
+    return this.delegateScan.getReadType();
+  }
+
+  @Override
+  long getMvccReadPoint() {
+    return this.delegateScan.getMvccReadPoint();
+  }
+
+  @Override
+  public boolean isNeedCursorResult() {
+    return this.delegateScan.isNeedCursorResult();
+  }
+
+  @Override
+  public byte[] getAttribute(String name) {
+    final byte[] attribute = this.delegateScan.getAttribute(name);
+    return Bytes.copy(attribute);
+  }
+
+  @Override
+  public Consistency getConsistency() {
+    return this.delegateScan.getConsistency();
+  }
+
+  @Override
+  public long getAttributeSize() {
+    return this.delegateScan.getAttributeSize();
+  }
+
+  @Override
+  public Map<String, byte[]> getAttributesMap() {
+    return Collections.unmodifiableMap(this.delegateScan.getAttributesMap());
+  }
+
+  @Override
+  public Boolean getLoadColumnFamiliesOnDemandValue() {
+    return this.delegateScan.getLoadColumnFamiliesOnDemandValue();
+  }
+
+  @Override
+  public int getPriority() {
+    return this.delegateScan.getPriority();
+  }
+
+  @Override
+  public Map<byte[], TimeRange> getColumnFamilyTimeRange() {
+    return Collections.unmodifiableMap(this.delegateScan.getColumnFamilyTimeRange());
+  }
+
+  @Override
+  public int getReplicaId() {
+    return this.delegateScan.getReplicaId();
+  }
+
+  @Override
+  public boolean doLoadColumnFamiliesOnDemand() {
+    return this.delegateScan.doLoadColumnFamiliesOnDemand();
+  }
+
+  @Override
+  public String getId() {
+    return this.delegateScan.getId();
+  }
+
+  @Override
+  public boolean isGetScan() {
+    return this.delegateScan.isGetScan();
+  }
+
+  @Override
+  public IsolationLevel getIsolationLevel() {
+    return this.delegateScan.getIsolationLevel();
+  }
+
+  @Override
+  public Authorizations getAuthorizations() throws DeserializationException {
+    return this.delegateScan.getAuthorizations();
+  }
+
+  @Override
+  public String toString(int maxCols) {
+    return this.delegateScan.toString(maxCols);
+  }
+
+  @Override
+  public String toString() {
+    return this.delegateScan.toString();
+  }
+
+  @Override
+  public Map<String, Object> toMap() {
+    return Collections.unmodifiableMap(this.delegateScan.toMap());
+  }
+
+  @Override
+  public String toJSON(int maxCols) throws IOException {
+    return this.delegateScan.toJSON(maxCols);
+  }
+
+  @Override
+  public String toJSON() throws IOException {
+    return this.delegateScan.toJSON();
+  }
+
+}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableScan.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableScan.java
new file mode 100644
index 0000000..4714936
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableScan.java
@@ -0,0 +1,392 @@
+/*
+ *
+ * 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.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.visibility.Authorizations;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Small tests for ImmutableScan
+ */
+@Category({ ClientTests.class, SmallTests.class })
+public class TestImmutableScan {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestImmutableScan.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestImmutableScan.class);
+
+  @Test
+  public void testScanCopyConstructor() throws Exception {
+    Scan scan = new Scan();
+
+    scan.addColumn(Bytes.toBytes("cf"), Bytes.toBytes("q"))
+      .setACL("test_user2", new Permission(Permission.Action.READ))
+      .setAllowPartialResults(true)
+      .setAsyncPrefetch(false)
+      .setAttribute("test_key", Bytes.toBytes("test_value"))
+      .setAuthorizations(new Authorizations("test_label"))
+      .setBatch(10)
+      .setCacheBlocks(false)
+      .setCaching(10)
+      .setConsistency(Consistency.TIMELINE)
+      .setFilter(new FilterList())
+      .setId("scan_copy_constructor")
+      .setIsolationLevel(IsolationLevel.READ_COMMITTED)
+      .setLimit(100)
+      .setLoadColumnFamiliesOnDemand(false)
+      .setMaxResultSize(100)
+      .setMaxResultsPerColumnFamily(1000)
+      .readVersions(9999)
+      .setMvccReadPoint(5)
+      .setNeedCursorResult(true)
+      .setPriority(1)
+      .setRaw(true)
+      .setReplicaId(3)
+      .setReversed(true)
+      .setRowOffsetPerColumnFamily(5)
+      .setRowPrefixFilter(Bytes.toBytes("row_"))
+      .setScanMetricsEnabled(true)
+      .setSmall(true)
+      .setReadType(Scan.ReadType.STREAM)
+      .withStartRow(Bytes.toBytes("row_1"))
+      .withStopRow(Bytes.toBytes("row_2"))
+      .setTimeRange(0, 13);
+
+    // create a copy of existing scan object
+    Scan scanCopy = new ImmutableScan(scan);
+
+    // validate fields of copied scan object match with the original scan object
+    assertArrayEquals(scan.getACL(), scanCopy.getACL());
+    assertEquals(scan.getAllowPartialResults(), scanCopy.getAllowPartialResults());
+    assertArrayEquals(scan.getAttribute("test_key"), scanCopy.getAttribute("test_key"));
+    assertEquals(scan.getAttributeSize(), scanCopy.getAttributeSize());
+    assertEquals(scan.getAttributesMap(), scanCopy.getAttributesMap());
+    assertEquals(scan.getAuthorizations().getLabels(), scanCopy.getAuthorizations().getLabels());
+    assertEquals(scan.getBatch(), scanCopy.getBatch());
+    assertEquals(scan.getCacheBlocks(), scanCopy.getCacheBlocks());
+    assertEquals(scan.getCaching(), scanCopy.getCaching());
+    assertEquals(scan.getConsistency(), scanCopy.getConsistency());
+    assertEquals(scan.getFamilies().length, scanCopy.getFamilies().length);
+    assertArrayEquals(scan.getFamilies()[0], scanCopy.getFamilies()[0]);
+    assertEquals(scan.getFamilyMap(), scanCopy.getFamilyMap());
+    assertEquals(scan.getFilter(), scanCopy.getFilter());
+    assertEquals(scan.getId(), scanCopy.getId());
+    assertEquals(scan.getIsolationLevel(), scanCopy.getIsolationLevel());
+    assertEquals(scan.getLimit(), scanCopy.getLimit());
+    assertEquals(scan.getLoadColumnFamiliesOnDemandValue(),
+      scanCopy.getLoadColumnFamiliesOnDemandValue());
+    assertEquals(scan.getMaxResultSize(), scanCopy.getMaxResultSize());
+    assertEquals(scan.getMaxResultsPerColumnFamily(), scanCopy.getMaxResultsPerColumnFamily());
+    assertEquals(scan.getMaxVersions(), scanCopy.getMaxVersions());
+    assertEquals(scan.getMvccReadPoint(), scanCopy.getMvccReadPoint());
+    assertEquals(scan.getPriority(), scanCopy.getPriority());
+    assertEquals(scan.getReadType(), scanCopy.getReadType());
+    assertEquals(scan.getReplicaId(), scanCopy.getReplicaId());
+    assertEquals(scan.getRowOffsetPerColumnFamily(), scanCopy.getRowOffsetPerColumnFamily());
+    assertArrayEquals(scan.getStartRow(), scanCopy.getStartRow());
+    assertArrayEquals(scan.getStopRow(), scanCopy.getStopRow());
+    assertEquals(scan.getTimeRange(), scanCopy.getTimeRange());
+    assertEquals(scan.getFingerprint(), scanCopy.getFingerprint());
+    assertEquals(scan.toMap(1), scanCopy.toMap(1));
+    assertEquals(scan.toString(2), scanCopy.toString(2));
+    assertEquals(scan.toJSON(2), scanCopy.toJSON(2));
+
+    LOG.debug("Compare all getters of scan and scanCopy.");
+    compareGetters(scan, scanCopy);
+
+    testUnmodifiableSetters(scanCopy);
+  }
+
+  private void testUnmodifiableSetters(Scan scanCopy) throws IOException {
+    try {
+      scanCopy.setFilter(Mockito.mock(Filter.class));
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setFilter", e.getMessage());
+    }
+    try {
+      scanCopy.addFamily(new byte[] { 0, 1 });
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to addFamily", e.getMessage());
+    }
+    try {
+      scanCopy.addColumn(new byte[] { 0, 1 }, new byte[] { 2, 3 });
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to addColumn", e.getMessage());
+    }
+    try {
+      scanCopy.setTimeRange(1L, 2L);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setTimeRange", e.getMessage());
+    }
+    try {
+      scanCopy.setTimestamp(1L);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setTimestamp", e.getMessage());
+    }
+    try {
+      scanCopy.setColumnFamilyTimeRange(new byte[] { 0 }, 1L, 2L);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setColumnFamilyTimeRange",
+        e.getMessage());
+    }
+    try {
+      scanCopy.withStopRow(new byte[] { 1, 2 });
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to withStopRow", e.getMessage());
+    }
+    try {
+      scanCopy.setRowPrefixFilter(new byte[] { 1, 2 });
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setRowPrefixFilter", e.getMessage());
+    }
+    try {
+      scanCopy.readAllVersions();
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to readAllVersions", e.getMessage());
+    }
+    try {
+      scanCopy.setBatch(1);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setBatch", e.getMessage());
+    }
+    try {
+      scanCopy.setRowOffsetPerColumnFamily(1);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setRowOffsetPerColumnFamily",
+        e.getMessage());
+    }
+    try {
+      scanCopy.setCaching(1);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setCaching",
+        e.getMessage());
+    }
+    try {
+      scanCopy.setLoadColumnFamiliesOnDemand(true);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setLoadColumnFamiliesOnDemand",
+        e.getMessage());
+    }
+    try {
+      scanCopy.setRaw(true);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setRaw", e.getMessage());
+    }
+    try {
+      scanCopy.setAuthorizations(new Authorizations("test"));
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setAuthorizations", e.getMessage());
+    }
+    try {
+      scanCopy.setACL("user1", new Permission(Permission.Action.READ));
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setACL", e.getMessage());
+    }
+    try {
+      scanCopy.setReplicaId(12);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setReplicaId", e.getMessage());
+    }
+    try {
+      scanCopy.setReadType(Scan.ReadType.STREAM);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setReadType", e.getMessage());
+    }
+    try {
+      scanCopy.setOneRowLimit();
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setOneRowLimit", e.getMessage());
+    }
+    try {
+      scanCopy.setNeedCursorResult(false);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setNeedCursorResult", e.getMessage());
+    }
+    try {
+      scanCopy.resetMvccReadPoint();
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to resetMvccReadPoint", e.getMessage());
+    }
+    try {
+      scanCopy.setMvccReadPoint(1L);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setMvccReadPoint", e.getMessage());
+    }
+    try {
+      scanCopy.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setIsolationLevel", e.getMessage());
+    }
+    try {
+      scanCopy.setPriority(10);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setPriority", e.getMessage());
+    }
+    try {
+      scanCopy.setConsistency(Consistency.TIMELINE);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setConsistency", e.getMessage());
+    }
+    try {
+      scanCopy.setCacheBlocks(true);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setCacheBlocks", e.getMessage());
+    }
+    try {
+      scanCopy.setAllowPartialResults(true);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setAllowPartialResults",
+        e.getMessage());
+    }
+    try {
+      scanCopy.setId("id");
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setId", e.getMessage());
+    }
+    try {
+      scanCopy.setMaxResultSize(100);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setMaxResultSize", e.getMessage());
+    }
+    try {
+      scanCopy.setMaxResultsPerColumnFamily(100);
+      throw new RuntimeException("Should not reach here");
+    } catch (UnsupportedOperationException e) {
+      assertEquals("ImmutableScan does not allow access to setMaxResultsPerColumnFamily",
+        e.getMessage());
+    }
+  }
+
+  private void compareGetters(Scan scan, Scan scanCopy) {
+    Method[] methods = Scan.class.getMethods();
+    for (Method method : methods) {
+      if (isGetter(method)) {
+        LOG.debug("Comparing return values of method: {}", method);
+        try {
+          Object obj1;
+          Object obj2;
+          switch (method.getName()) {
+            case "toMap": {
+              if (method.getParameterCount() == 1) {
+                obj1 = method.invoke(scan, 2);
+                obj2 = method.invoke(scanCopy, 2);
+                break;
+              }
+            }
+            case "getAttribute": {
+              if (method.getParameterCount() == 1) {
+                obj1 = method.invoke(scan, "acl");
+                obj2 = method.invoke(scanCopy, "acl");
+                break;
+              }
+            }
+            case "toString": {
+              if (method.getParameterCount() == 1) {
+                obj1 = method.invoke(scan, 25);
+                obj2 = method.invoke(scanCopy, 25);
+                break;
+              }
+            }
+            case "toJSON": {
+              if (method.getParameterCount() == 1) {
+                obj1 = method.invoke(scan, 25);
+                obj2 = method.invoke(scanCopy, 25);
+                break;
+              }
+            }
+            default: {
+              obj1 = method.invoke(scan);
+              obj2 = method.invoke(scanCopy);
+            }
+          }
+          if (obj1 instanceof Map && obj2 instanceof Map) {
+            obj1 = Collections.unmodifiableMap((Map<?, ?>) obj1);
+          }
+          if (!EqualsBuilder.reflectionEquals(obj1, obj2)) {
+            throw new AssertionError("Method " + method + " does not return equal values");
+          }
+        } catch (IllegalAccessException | InvocationTargetException | IllegalArgumentException e) {
+          throw new AssertionError("Error invoking method " + method, e);
+        }
+      }
+    }
+  }
+
+  private static boolean isGetter(Method method) {
+    if ("hashCode".equals(method.getName()) || "equals".equals(method.getName())
+        || method.getName().startsWith("set")) {
+      return false;
+    }
+    return !void.class.equals(method.getReturnType())
+      && !Scan.class.equals(method.getReturnType());
+  }
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
index 81a43c6..ac9980b0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.client.ImmutableScan;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -42,16 +43,13 @@ public class CustomizedScanInfoBuilder implements ScanOptions {
 
   public CustomizedScanInfoBuilder(ScanInfo scanInfo) {
     this.scanInfo = scanInfo;
-    this.scan = new Scan();
+    this.scan = new ImmutableScan(new Scan());
   }
+
   public CustomizedScanInfoBuilder(ScanInfo scanInfo, Scan scan) {
     this.scanInfo = scanInfo;
     //copy the scan so no coproc using this ScanOptions can alter the "real" scan
-    try {
-      this.scan = new Scan(scan);
-    } catch (IOException e) {
-      throw new AssertionError("Scan should not throw IOException", e);
-    }
+    this.scan = new ImmutableScan(scan);
   }
 
   @Override
@@ -109,11 +107,7 @@ public class CustomizedScanInfoBuilder implements ScanOptions {
 
   @Override
   public Scan getScan() {
-    try {
-      return new Scan(scan);
-    } catch(IOException e) {
-      throw new AssertionError("Scan should not throw IOException anymore", e);
-    }
+    return scan;
   }
 
 }