You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by om...@apache.org on 2017/10/18 22:17:46 UTC

[1/2] orc git commit: ORC-245. Create framework for data masking.

Repository: orc
Updated Branches:
  refs/heads/master aae40f123 -> d5a6c49bf


http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/test/org/apache/orc/impl/mask/TestRedactMask.java
----------------------------------------------------------------------
diff --git a/java/core/src/test/org/apache/orc/impl/mask/TestRedactMask.java b/java/core/src/test/org/apache/orc/impl/mask/TestRedactMask.java
new file mode 100644
index 0000000..ff4ef35
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/impl/mask/TestRedactMask.java
@@ -0,0 +1,247 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.junit.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Timestamp;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestRedactMask {
+
+  @Test
+  public void testSimpleReplaceLongDigits() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory("Xx7");
+    assertEquals(7, mask.maskLong(0));
+    assertEquals(7, mask.maskLong(9));
+    assertEquals(-7, mask.maskLong(-9));
+    assertEquals(-7, mask.maskLong(-1));
+    assertEquals(77, mask.maskLong(10));
+    assertEquals(-77, mask.maskLong(-10));
+    assertEquals(7_777_777_777_777_777_777L,
+        mask.maskLong(Long.MAX_VALUE));
+    assertEquals(-7_777_777_777_777_777_777L,
+        mask.maskLong(Long.MIN_VALUE + 1));
+    assertEquals(-7_777_777_777_777_777_777L,
+        mask.maskLong(Long.MIN_VALUE));
+  }
+
+  @Test
+  public void testPow10ReplaceLongDigits() throws Exception {
+    for(int digit=0; digit < 10; ++digit) {
+      RedactMaskFactory mask = new RedactMaskFactory("Xx" + digit);
+      long expected = digit;
+      long input = 1;
+      for(int i=0; i < 19; ++i) {
+        // 9_999_999_999_999_999_999 is bigger than 2**63, so it overflows.
+        // The routine uses one less digit for that case.
+        if (i == 18 && digit == 9) {
+          expected = 999_999_999_999_999_999L;
+        }
+        assertEquals("digit " + digit + " value " + input, expected,
+            mask.maskLong(input));
+        assertEquals("digit " + digit + " value " + (5 * input), expected,
+            mask.maskLong(5 * input));
+        assertEquals("digit " + digit + " value " + (9 * input), expected,
+            mask.maskLong(9 * input));
+        expected = expected * 10 + digit;
+        input *= 10;
+      }
+    }
+  }
+
+  @Test
+  public void testSimpleReplaceDoubleDigits() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory("Xx7");
+    assertEquals(7.77777, mask.maskDouble(0.0), 0.000001);
+    assertEquals(7.77777, mask.maskDouble(9.9), 0.000001);
+    assertEquals(-7.77777, mask.maskDouble(-9.9), 0.000001);
+    assertEquals(-7.77777, mask.maskDouble(-1.0), 0.000001);
+    assertEquals(77.7777, mask.maskDouble(10.0), 0.000001);
+    assertEquals(-77.7777, mask.maskDouble(-10.0), 0.000001);
+    assertEquals(7_777_770_000_000_000_000.0,
+        mask.maskDouble(Long.MAX_VALUE), 0.000001);
+    assertEquals(-7_777_770_000_000_000_000.0,
+        mask.maskDouble(Long.MIN_VALUE), 0.000001);
+    assertEquals(7.77777e-308,
+        mask.maskDouble(Double.MIN_NORMAL), 1e-310);
+    assertEquals(7.77777e307,
+        mask.maskDouble(Double.MAX_VALUE), 1e299);
+
+    // change to mask of 1
+    mask = new RedactMaskFactory("Xx1");
+    assertEquals(-1.11111e-308,
+        mask.maskDouble(-Double.MIN_NORMAL), 1e-310);
+
+    // change to mask of 9
+    mask = new RedactMaskFactory();
+    assertEquals(-9.99999e307,
+        mask.maskDouble(-Double.MAX_VALUE), 1e299);
+  }
+
+  @Test
+  public void testSimpleMaskTimestamp() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory();
+    Timestamp ts = Timestamp.valueOf("2011-10-02 18:48:05.123456");
+    assertEquals("2011-01-01 00:00:00.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    ts = Timestamp.valueOf("2012-02-28 01:23:45");
+    assertEquals("2012-01-01 00:00:00.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    ts = Timestamp.valueOf("2017-05-18 01:23:45");
+    assertEquals("2017-01-01 00:00:00.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "2000 _ _ 15 0 _");
+    assertEquals("2000-05-18 15:00:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "2000 _ _ 15 0 _");
+    assertEquals("2000-05-18 15:00:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "2007 _ _ _ _ _");
+    assertEquals("2007-05-18 01:23:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "_ 7 _ _ _ _");
+    assertEquals("2017-07-18 01:23:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "_ _ 7 _ _ _");
+    assertEquals("2017-05-07 01:23:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "_ _ _ 7 _ _");
+    assertEquals("2017-05-18 07:23:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "_ _ _ _ 7 _");
+    assertEquals("2017-05-18 01:07:45.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+    mask = new RedactMaskFactory("", "_ _ _ _ _ 7");
+    assertEquals("2017-05-18 01:23:07.0",
+        new Timestamp(mask.maskTime(ts.getTime())).toString());
+  }
+
+  @Test
+  public void testSimpleMaskDate() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory();
+    DateWritable date = new DateWritable(Date.valueOf("1965-03-12"));
+    assertEquals("1965-01-01",
+        new DateWritable(mask.maskDate(date.getDays())).toString());
+    mask = new RedactMaskFactory("", "2000 _ _");
+    assertEquals("2000-03-12",
+        new DateWritable(mask.maskDate(date.getDays())).toString());
+    mask = new RedactMaskFactory(new String[]{"", "_ 7 _"});
+    assertEquals("1965-07-12",
+        new DateWritable(mask.maskDate(date.getDays())).toString());
+    mask = new RedactMaskFactory("", "_ _ 7");
+    assertEquals("1965-03-07",
+        new DateWritable(mask.maskDate(date.getDays())).toString());
+    date = new DateWritable(Date.valueOf("2017-09-20"));
+    assertEquals("2017-09-07",
+        new DateWritable(mask.maskDate(date.getDays())).toString());
+  }
+
+  @Test
+  public void testSimpleMaskDecimal() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory("Xx7");
+    assertEquals(new HiveDecimalWritable("777.777"),
+        mask.maskDecimal(new HiveDecimalWritable("123.456")));
+    // test removal of leading and  trailing zeros.
+    assertEquals(new HiveDecimalWritable("777777777777777777.7777"),
+        mask.maskDecimal(new HiveDecimalWritable("0123456789123456789.01230")));
+  }
+
+  @Test
+  public void testReplacements() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory("1234567890");
+    assertEquals("1".codePointAt(0), mask.getReplacement("X".codePointAt(0)));
+    assertEquals("2".codePointAt(0), mask.getReplacement("x".codePointAt(0)));
+    assertEquals("3".codePointAt(0), mask.getReplacement("0".codePointAt(0)));
+    assertEquals("4".codePointAt(0), mask.getReplacement("$".codePointAt(0)));
+    assertEquals("5".codePointAt(0), mask.getReplacement(".".codePointAt(0)));
+    assertEquals("6".codePointAt(0), mask.getReplacement(" ".codePointAt(0)));
+    assertEquals("7".codePointAt(0), mask.getReplacement("ה".codePointAt(0)));
+    assertEquals("8".codePointAt(0), mask.getReplacement("ी".codePointAt(0)));
+    assertEquals("9".codePointAt(0), mask.getReplacement("ↂ".codePointAt(0)));
+    assertEquals("0".codePointAt(0), mask.getReplacement("\u06DD".codePointAt(0)));
+    mask = new RedactMaskFactory();
+    assertEquals("_".codePointAt(0), mask.getReplacement(" ".codePointAt(0)));
+  }
+
+  @Test
+  public void testStringMasking() throws Exception {
+    RedactMaskFactory mask = new RedactMaskFactory();
+    BytesColumnVector source = new BytesColumnVector();
+    BytesColumnVector target = new BytesColumnVector();
+    target.reset();
+    byte[] input = "Mary had 1 little lamb!!".getBytes(StandardCharsets.UTF_8);
+    source.setRef(0, input, 0, input.length);
+
+    // Set a 4 byte chinese character (U+2070E), which is letter other
+    input = "\uD841\uDF0E".getBytes(StandardCharsets.UTF_8);
+    source.setRef(1, input, 0, input.length);
+    for(int r=0; r < 2; ++r) {
+      mask.maskString(source, r, target);
+    }
+    assertEquals("Xxxx xxx 9 xxxxxx xxxx..", new String(target.vector[0],
+        target.start[0], target.length[0], StandardCharsets.UTF_8));
+    assertEquals("ª", new String(target.vector[1],
+        target.start[1], target.length[1], StandardCharsets.UTF_8));
+  }
+
+  @Test
+  public void testStringMaskBufferOverflow() throws Exception {
+    // set upper and lower letters to replace with 4 byte replacements
+    // (U+267CC and U+28CCA)
+    RedactMaskFactory mask = new RedactMaskFactory("\uD859\uDFCC\uD863\uDCCA");
+    BytesColumnVector source = new BytesColumnVector();
+    BytesColumnVector target = new BytesColumnVector();
+    target.reset();
+
+    // Set the input to 1024 copies of the input string.
+    // input is 14 bytes * 1024 = 14336 bytes
+    // output is (4 * 12 + 1 * 2) * 1024 = 51200 bytes
+    byte[] input = "text overflow."
+        .getBytes(StandardCharsets.UTF_8);
+    for(int r=0; r < 1024; ++r) {
+      source.setRef(r, input, 0, input.length);
+    }
+    for(int r=0; r < 1024; ++r) {
+      mask.maskString(source, r, target);
+    }
+
+    // should have doubled twice to 64k
+    assertEquals(64*1024, target.getValPreallocatedBytes().length);
+
+    // Make sure all of the translations are correct
+    String expected ="\uD863\uDCCA\uD863\uDCCA\uD863\uDCCA\uD863\uDCCA" +
+        " \uD863\uDCCA\uD863\uDCCA\uD863\uDCCA\uD863\uDCCA" +
+        "\uD863\uDCCA\uD863\uDCCA\uD863\uDCCA\uD863\uDCCA.";
+    for(int r=0; r < 1024; ++r) {
+      assertEquals("r = " + r, expected,
+          new String(target.vector[r], target.start[r], target.length[r],
+              StandardCharsets.UTF_8));
+    }
+
+    // Make sure that the target keeps the larger output buffer.
+    target.reset();
+    assertEquals(64*1024, target.getValPreallocatedBytes().length);
+  }
+}


[2/2] orc git commit: ORC-245. Create framework for data masking.

Posted by om...@apache.org.
ORC-245. Create framework for data masking.

Fixes #174

Signed-off-by: Owen O'Malley <om...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/orc/repo
Commit: http://git-wip-us.apache.org/repos/asf/orc/commit/d5a6c49b
Tree: http://git-wip-us.apache.org/repos/asf/orc/tree/d5a6c49b
Diff: http://git-wip-us.apache.org/repos/asf/orc/diff/d5a6c49b

Branch: refs/heads/master
Commit: d5a6c49bf994ee9b07bad699fdcb2af8b7dd4afc
Parents: aae40f1
Author: Owen O'Malley <om...@apache.org>
Authored: Tue Oct 10 14:58:33 2017 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Wed Oct 18 15:15:14 2017 -0700

----------------------------------------------------------------------
 java/core/pom.xml                               |   5 +
 java/core/src/java/org/apache/orc/DataMask.java | 124 +++
 .../apache/orc/impl/mask/DecimalIdentity.java   |  55 ++
 .../apache/orc/impl/mask/DoubleIdentity.java    |  50 ++
 .../org/apache/orc/impl/mask/ListIdentity.java  |  68 ++
 .../org/apache/orc/impl/mask/LongIdentity.java  |  50 ++
 .../org/apache/orc/impl/mask/MapIdentity.java   |  75 ++
 .../org/apache/orc/impl/mask/MaskFactory.java   | 101 +++
 .../org/apache/orc/impl/mask/MaskProvider.java  |  39 +
 .../org/apache/orc/impl/mask/NullifyMask.java   |  35 +
 .../apache/orc/impl/mask/RedactMaskFactory.java | 857 +++++++++++++++++++
 .../apache/orc/impl/mask/StructIdentity.java    |  74 ++
 .../apache/orc/impl/mask/TimestampIdentity.java |  52 ++
 .../org/apache/orc/impl/mask/UnionIdentity.java |  66 ++
 .../services/org.apache.orc.DataMask$Provider   |  15 +
 .../org/apache/orc/impl/mask/TestDataMask.java  | 177 ++++
 .../apache/orc/impl/mask/TestRedactMask.java    | 247 ++++++
 17 files changed, 2090 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/java/core/pom.xml b/java/core/pom.xml
index 32b2c15..e2973eb 100644
--- a/java/core/pom.xml
+++ b/java/core/pom.xml
@@ -95,6 +95,11 @@
   <build>
     <sourceDirectory>${basedir}/src/java</sourceDirectory>
     <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <resources>
+      <resource>
+	<directory>${basedir}/src/resources</directory>
+      </resource>
+    </resources>
     <testResources>
       <testResource>
         <directory>${basedir}/src/test/resources</directory>

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/DataMask.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/DataMask.java b/java/core/src/java/org/apache/orc/DataMask.java
new file mode 100644
index 0000000..df6ae67
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/DataMask.java
@@ -0,0 +1,124 @@
+/*
+ * 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.orc;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+
+import java.util.ServiceLoader;
+
+/**
+ * The API for masking data during column encryption for ORC.
+ *
+ * They apply to an individual column (via ColumnVector) instead of a
+ * VectorRowBatch.
+ *
+ */
+public interface DataMask {
+
+  /**
+   * The standard DataMasks can be created using this short cut.
+   *
+   * For example, DataMask.Standard.NULLIFY.build(schema) will build a
+   * nullify DataMask.
+   */
+  enum Standard {
+    NULLIFY("nullify"),
+    REDACT("redact");
+
+    Standard(String name) {
+      this.name = name;
+    }
+
+    private final String name;
+
+    public String getName() {
+      return name;
+    }
+
+    public DataMask build(TypeDescription schema, String... params) {
+      return Factory.build(name, schema, params);
+    }
+  }
+
+  /**
+   * Mask the given range of values
+   * @param original the original input data
+   * @param masked the masked output data
+   * @param start the first data element to mask
+   * @param length the number of data elements to mask
+   */
+  void maskData(ColumnVector original, ColumnVector masked,
+                int start, int length);
+
+
+  /**
+   * Providers can provide one or more kinds of data masks.
+   * Because they are discovered using a service loader, they may be added
+   * by third party jars.
+   */
+  interface Provider {
+    /**
+     * Build a mask with the given parameters.
+     * @param name the kind of masking
+     * @param schema the type of the field
+     * @param params the list of parameters with the name in params[0]
+     * @return the new data mask or null if this name is unknown
+     */
+    DataMask build(String name, TypeDescription schema, String... params);
+  }
+
+  /**
+   * To create a DataMask, the users should come through this API.
+   *
+   * It supports extension via additional DataMask.Provider implementations
+   * that are accessed through Java's ServiceLoader API.
+   */
+  class Factory {
+    private static final ServiceLoader<Provider> LOADER =
+        ServiceLoader.load(Provider.class);
+
+    /**
+     * Build a new DataMask instance.
+     * @param name the name of the mask
+     * @param schema the type of the field
+     * @param params a list of parameters to the mask
+     * @return a new DataMask
+     * @throws IllegalArgumentException if no such kind of data mask was found
+     *
+     * @see org.apache.orc.impl.mask.MaskProvider for the standard provider
+     */
+    public static DataMask build(String name,
+                           TypeDescription schema,
+                           String... params) {
+      for(Provider provider: LOADER) {
+        DataMask result = provider.build(name, schema, params);
+        if (result != null) {
+          return result;
+        }
+      }
+      StringBuilder msg = new StringBuilder();
+      msg.append("Can't find data mask - ");
+      msg.append(name);
+      for(int i=0; i < params.length; ++i) {
+        msg.append(", ");
+        msg.append(params[i]);
+      }
+      throw new IllegalArgumentException(msg.toString());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/DecimalIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/DecimalIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/DecimalIdentity.java
new file mode 100644
index 0000000..31f1ffd
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/DecimalIdentity.java
@@ -0,0 +1,55 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * An identity data mask for decimal types.
+ */
+public class DecimalIdentity implements DataMask {
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    DecimalColumnVector target = (DecimalColumnVector) masked;
+    DecimalColumnVector source = (DecimalColumnVector) original;
+    target.scale = source.scale;
+    target.precision = source.precision;
+    target.isRepeating = source.isRepeating;
+    target.noNulls = source.noNulls;
+    if (source.isRepeating) {
+      target.vector[0].set(source.vector[0]);
+      target.isNull[0] = source.isNull[0];
+    } else if (source.noNulls) {
+      for(int r = start; r < start + length; ++r) {
+        target.vector[r].set(source.vector[r]);
+      }
+    } else {
+      for(int r = start; r < start + length; ++r) {
+        target.isNull[r] = source.isNull[r];
+        if (!target.isNull[r]) {
+          target.vector[r].set(source.vector[r]);
+        }
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/DoubleIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/DoubleIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/DoubleIdentity.java
new file mode 100644
index 0000000..2660a6c
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/DoubleIdentity.java
@@ -0,0 +1,50 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * An identity data mask for floating point types.
+ */
+public class DoubleIdentity implements DataMask {
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    DoubleColumnVector target = (DoubleColumnVector) masked;
+    DoubleColumnVector source = (DoubleColumnVector) original;
+    target.isRepeating = source.isRepeating;
+    target.noNulls = source.noNulls;
+    if (source.isRepeating) {
+      target.vector[0] = source.vector[0];
+      target.isNull[0] = source.isNull[0];
+    } else if (source.noNulls) {
+      for(int r = start; r < start + length; ++r) {
+        target.vector[r] = source.vector[r];
+      }
+    } else {
+      for(int r = start; r < start + length; ++r) {
+        target.isNull[r] = source.isNull[r];
+        target.vector[r] = source.vector[r];
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/ListIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/ListIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/ListIdentity.java
new file mode 100644
index 0000000..7592802
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/ListIdentity.java
@@ -0,0 +1,68 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * A data mask for list types that applies the given masks to its
+ * children, but doesn't mask at this level.
+ */
+public class ListIdentity implements DataMask {
+  private final DataMask child;
+
+  ListIdentity(DataMask[] child) {
+    this.child = child[0];
+  }
+
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start, int length) {
+    ListColumnVector source = (ListColumnVector) original;
+    ListColumnVector target = (ListColumnVector) masked;
+    target.noNulls = source.noNulls;
+    target.isRepeating = source.isRepeating;
+    if (source.isRepeating) {
+      if (!source.noNulls && source.isNull[0]) {
+        target.isNull[0] = true;
+      } else {
+        target.lengths[0] = source.lengths[0];
+        child.maskData(source.child, target.child, (int) source.offsets[0],
+            (int) source.lengths[0]);
+      }
+    } else if (source.noNulls) {
+        for(int r=start; r < start+length; ++r) {
+          target.offsets[r] = source.offsets[r];
+          target.lengths[r] = source.lengths[r];
+          child.maskData(source.child, target.child, (int) target.offsets[r],
+              (int) target.lengths[r]);
+        }
+    } else {
+      for(int r=start; r < start+length; ++r) {
+        target.isNull[r] = source.isNull[r];
+        if (!source.isNull[r]) {
+          target.offsets[r] = source.offsets[r];
+          target.lengths[r] = source.lengths[r];
+          child.maskData(source.child, target.child, (int) target.offsets[r],
+              (int) target.lengths[r]);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/LongIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/LongIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/LongIdentity.java
new file mode 100644
index 0000000..69a0718
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/LongIdentity.java
@@ -0,0 +1,50 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * An identity data mask for integer types.
+ */
+public class LongIdentity implements DataMask {
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    LongColumnVector target = (LongColumnVector) masked;
+    LongColumnVector source = (LongColumnVector) original;
+    target.isRepeating = source.isRepeating;
+    target.noNulls = source.noNulls;
+    if (original.isRepeating) {
+      target.vector[0] = source.vector[0];
+      target.isNull[0] = source.isNull[0];
+    } else if (source.noNulls) {
+      for(int r = start; r < start + length; ++r) {
+        target.vector[r] = source.vector[r];
+      }
+    } else {
+      for(int r = start; r < start + length; ++r) {
+        target.vector[r] = source.vector[r];
+        target.isNull[r] = source.isNull[r];
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/MapIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/MapIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/MapIdentity.java
new file mode 100644
index 0000000..43a1aea
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/MapIdentity.java
@@ -0,0 +1,75 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * A data mask for map types that applies the given masks to its
+ * children, but doesn't mask at this level.
+ */
+public class MapIdentity implements DataMask {
+  private final DataMask keyMask;
+  private final DataMask valueMask;
+
+  MapIdentity(DataMask[] children) {
+    this.keyMask = children[0];
+    this.valueMask = children[1];
+  }
+
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    MapColumnVector source = (MapColumnVector) original;
+    MapColumnVector target = (MapColumnVector) masked;
+    target.isRepeating = source.isRepeating;
+    target.noNulls = source.noNulls;
+    if (source.isRepeating) {
+      target.isNull[0] = source.isNull[0];
+      if (source.noNulls || !source.isNull[0]) {
+        target.lengths[0] = source.lengths[0];
+        keyMask.maskData(source.keys, target.keys, (int) source.offsets[0],
+            (int) source.lengths[0]);
+        valueMask.maskData(source.values, target.values, (int) source.offsets[0],
+            (int) source.lengths[0]);      }
+    } else if (source.noNulls) {
+      for(int r=start; r < start+length; ++r) {
+        target.offsets[r] = source.offsets[r];
+        target.lengths[r] = source.lengths[r];
+        keyMask.maskData(source.keys, target.keys, (int) target.offsets[r],
+            (int) target.lengths[r]);
+        valueMask.maskData(source.values, target.values, (int) target.offsets[r],
+            (int) target.lengths[r]);
+      }
+    } else {
+      for(int r=start; r < start+length; ++r) {
+        target.isNull[r] = source.isNull[r];
+        if (!source.isNull[r]) {
+          target.offsets[r] = source.offsets[r];
+          target.lengths[r] = source.lengths[r];
+          keyMask.maskData(source.keys, target.keys, (int) target.offsets[r],
+              (int) target.lengths[r]);
+          valueMask.maskData(source.values, target.values, (int) target.offsets[r],
+              (int) target.lengths[r]);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/MaskFactory.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/MaskFactory.java b/java/core/src/java/org/apache/orc/impl/mask/MaskFactory.java
new file mode 100644
index 0000000..9a77c4a
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/MaskFactory.java
@@ -0,0 +1,101 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.orc.DataMask;
+import org.apache.orc.TypeDescription;
+
+import java.util.List;
+
+/**
+ * A mask factory framework that automatically builds a recursive mask.
+ * The subclass defines how to mask the primitive types and the factory
+ * builds a recursive tree of data masks that matches the schema tree.
+ */
+public abstract class MaskFactory {
+
+  protected abstract DataMask buildBooleanMask(TypeDescription schema);
+  protected abstract DataMask buildLongMask(TypeDescription schema);
+  protected abstract DataMask buildDecimalMask(TypeDescription schema);
+  protected abstract DataMask buildDoubleMask(TypeDescription schema);
+  protected abstract DataMask buildStringMask(TypeDescription schema);
+  protected abstract DataMask buildDateMask(TypeDescription schema);
+  protected abstract DataMask buildTimestampMask(TypeDescription schema);
+  protected abstract DataMask buildBinaryMask(TypeDescription schema);
+
+  public DataMask build(TypeDescription schema) {
+    switch(schema.getCategory()) {
+      case BOOLEAN:
+        return buildBooleanMask(schema);
+      case BYTE:
+      case SHORT:
+      case INT:
+      case LONG:
+        return buildLongMask(schema);
+      case FLOAT:
+      case DOUBLE:
+        return buildDoubleMask(schema);
+      case DECIMAL:
+        return buildDecimalMask(schema);
+      case STRING:
+      case CHAR:
+      case VARCHAR:
+        return buildStringMask(schema);
+      case TIMESTAMP:
+        return buildTimestampMask(schema);
+      case DATE:
+        return buildDateMask(schema);
+      case BINARY:
+        return buildBinaryMask(schema);
+      case UNION:
+        return buildUnionMask(schema);
+      case STRUCT:
+        return buildStructMask(schema);
+      case LIST:
+        return buildListMask(schema);
+      case MAP:
+        return buildMapMask(schema);
+      default:
+        throw new IllegalArgumentException("Unhandled type " + schema);
+    }
+  }
+
+  protected DataMask[] buildChildren(List<TypeDescription> children) {
+    DataMask[] result = new DataMask[children.size()];
+    for(int i = 0; i < result.length; ++i) {
+      result[i] = build(children.get(i));
+    }
+    return result;
+  }
+
+  protected DataMask buildStructMask(TypeDescription schema) {
+    return new StructIdentity(buildChildren(schema.getChildren()));
+  }
+
+  DataMask buildListMask(TypeDescription schema) {
+    return new ListIdentity(buildChildren(schema.getChildren()));
+  }
+
+  DataMask buildMapMask(TypeDescription schema) {
+    return new MapIdentity(buildChildren(schema.getChildren()));
+  }
+
+  DataMask buildUnionMask(TypeDescription schema) {
+    return new UnionIdentity(buildChildren(schema.getChildren()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/MaskProvider.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/MaskProvider.java b/java/core/src/java/org/apache/orc/impl/mask/MaskProvider.java
new file mode 100644
index 0000000..58f7b38
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/MaskProvider.java
@@ -0,0 +1,39 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.orc.DataMask;
+import org.apache.orc.TypeDescription;
+
+/**
+ * The Provider for all of the built-in data masks.
+ */
+public class MaskProvider implements DataMask.Provider {
+
+  @Override
+  public DataMask build(String name,
+                        TypeDescription schema,
+                        String... params) {
+    if (name.equals(DataMask.Standard.NULLIFY.getName())) {
+      return new NullifyMask();
+    } else if (name.equals(DataMask.Standard.REDACT.getName())) {
+      return new RedactMaskFactory(params).build(schema);
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/NullifyMask.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/NullifyMask.java b/java/core/src/java/org/apache/orc/impl/mask/NullifyMask.java
new file mode 100644
index 0000000..1ff8bc1
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/NullifyMask.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * Masking routine that converts every value to NULL.
+ */
+public class NullifyMask implements DataMask {
+
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked,
+                       int start, int length) {
+    masked.noNulls = false;
+    masked.isRepeating = true;
+    masked.isNull[0] = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/RedactMaskFactory.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/RedactMaskFactory.java b/java/core/src/java/org/apache/orc/impl/mask/RedactMaskFactory.java
new file mode 100644
index 0000000..37a5ba4
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/RedactMaskFactory.java
@@ -0,0 +1,857 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.DataMask;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/**
+ * Masking strategy that hides most string and numeric values based on unicode
+ * character categories.
+ *
+ * Masking Parameters:
+ *   character replacements: string of 10 characters one per group below
+ *     letter, upper case (default X)
+ *     letter, lower case (default x)
+ *     number, digit      (default 9)
+ *     symbol             (default $)
+ *     punctuation        (default .)
+ *     separator          (default no masking)
+ *     letter, other      (default ª)
+ *     mark               (default ः)
+ *     number, other      (default ²)
+ *     other              (default ۝)
+ *
+ *   time replacements: string of 6 numbers or _ one per field below
+ *     year (0 to 4000, default no masking)
+ *     month (1 to 12, default 1)
+ *     date (1 to 31, default 1)
+ *     hour (0 to 23, default 0)
+ *     minute (0 to 59, default 0)
+ *     second (0 to 59, default 0)
+ *
+ * Parameters use "_" for preserve original.
+ */
+public class RedactMaskFactory extends MaskFactory {
+
+  /**
+   * The value to indicate that the value should be preserved.
+   */
+  private static final int UNMASKED_CHAR = "_".codePointAt(0);
+  private static final int UNMASKED_DATE = -1;
+
+  // The default replacements for each character category.
+  // I picked a character in the same category so that the masking is
+  // idempotent. For non-ascii characters, I mostly picked the first example.
+  private static final int DEFAULT_LETTER_UPPER = "X".codePointAt(0);
+  private static final int DEFAULT_LETTER_LOWER = "x".codePointAt(0);
+  private static final int DEFAULT_NUMBER_DIGIT = 9;
+  private static final int DEFAULT_NUMBER_DIGIT_CP =
+      Integer.toString(DEFAULT_NUMBER_DIGIT).codePointAt(0);
+  private static final int DEFAULT_SYMBOL = "$".codePointAt(0);
+  private static final int DEFAULT_PUNCTUATION = ".".codePointAt(0);
+  private static final int DEFAULT_SEPARATOR = UNMASKED_CHAR;
+  private static final int DEFAULT_LETTER_OTHER = "\u00AA".codePointAt(0);
+  private static final int DEFAULT_MARK = "\u0903".codePointAt(0);
+  private static final int DEFAULT_NUMBER_OTHER = "\u00B2".codePointAt(0);
+  private static final int DEFAULT_OTHER = "\u06DD".codePointAt(0);
+
+  // The replacement codepoint for each character category. We use codepoints
+  // here so that we don't have to worry about handling long UTF characters
+  // as special cases.
+  private final int UPPPER_REPLACEMENT;
+  private final int LOWER_REPLACEMENT;
+  private final int OTHER_LETTER_REPLACEMENT;
+  private final int MARK_REPLACEMENT;
+  private final int DIGIT_CP_REPLACEMENT;
+  private final int OTHER_NUMBER_REPLACEMENT;
+  private final int SYMBOL_REPLACEMENT;
+  private final int PUNCTUATION_REPLACEMENT;
+  private final int SEPARATOR_REPLACEMENT;
+  private final int OTHER_REPLACEMENT;
+
+  // numeric replacement
+  private final int DIGIT_REPLACEMENT;
+
+  // time replacement
+  private final int YEAR_REPLACEMENT;
+  private final int MONTH_REPLACEMENT;
+  private final int DATE_REPLACEMENT;
+  private final int HOUR_REPLACEMENT;
+  private final int MINUTE_REPLACEMENT;
+  private final int SECOND_REPLACEMENT;
+  private final boolean maskDate;
+  private final boolean maskTimestamp;
+
+  public RedactMaskFactory(String... params) {
+    ByteBuffer param = params.length < 1 ? ByteBuffer.allocate(0) :
+        ByteBuffer.wrap(params[0].getBytes(StandardCharsets.UTF_8));
+    UPPPER_REPLACEMENT = getNextCodepoint(param, DEFAULT_LETTER_UPPER);
+    LOWER_REPLACEMENT = getNextCodepoint(param, DEFAULT_LETTER_LOWER);
+    DIGIT_CP_REPLACEMENT = getNextCodepoint(param, DEFAULT_NUMBER_DIGIT_CP);
+    DIGIT_REPLACEMENT = getReplacementDigit(DIGIT_CP_REPLACEMENT);
+    SYMBOL_REPLACEMENT = getNextCodepoint(param, DEFAULT_SYMBOL);
+    PUNCTUATION_REPLACEMENT = getNextCodepoint(param, DEFAULT_PUNCTUATION);
+    SEPARATOR_REPLACEMENT = getNextCodepoint(param, DEFAULT_SEPARATOR);
+    OTHER_LETTER_REPLACEMENT = getNextCodepoint(param, DEFAULT_LETTER_OTHER);
+    MARK_REPLACEMENT = getNextCodepoint(param, DEFAULT_MARK);
+    OTHER_NUMBER_REPLACEMENT = getNextCodepoint(param, DEFAULT_NUMBER_OTHER);
+    OTHER_REPLACEMENT = getNextCodepoint(param, DEFAULT_OTHER);
+    String[] timeParams;
+    if (params.length < 2) {
+      timeParams = null;
+    } else {
+      timeParams = params[1].split("\\W+");
+    }
+    YEAR_REPLACEMENT = getDateParam(timeParams, 0, UNMASKED_DATE, 4000);
+    MONTH_REPLACEMENT = getDateParam(timeParams, 1, 1, 12);
+    DATE_REPLACEMENT = getDateParam(timeParams, 2, 1, 31);
+    HOUR_REPLACEMENT = getDateParam(timeParams, 3, 0, 23);
+    MINUTE_REPLACEMENT = getDateParam(timeParams, 4, 0, 59);
+    SECOND_REPLACEMENT = getDateParam(timeParams, 5, 0, 59);
+    maskDate = (YEAR_REPLACEMENT != UNMASKED_DATE) ||
+        (MONTH_REPLACEMENT != UNMASKED_DATE) ||
+        (DATE_REPLACEMENT != UNMASKED_DATE);
+    maskTimestamp = maskDate || (HOUR_REPLACEMENT != UNMASKED_DATE) ||
+        (MINUTE_REPLACEMENT != UNMASKED_DATE) ||
+        (SECOND_REPLACEMENT != UNMASKED_DATE);
+  }
+
+  @Override
+  protected DataMask buildBooleanMask(TypeDescription schema) {
+    if (DIGIT_CP_REPLACEMENT == UNMASKED_CHAR) {
+      return new LongIdentity();
+    } else {
+      return new BooleanRedactConverter();
+    }
+  }
+
+  @Override
+  protected DataMask buildLongMask(TypeDescription schema) {
+    if (DIGIT_CP_REPLACEMENT == UNMASKED_CHAR) {
+      return new LongIdentity();
+    } else {
+      return new LongRedactConverter(schema.getCategory());
+    }
+  }
+
+  @Override
+  protected DataMask buildDecimalMask(TypeDescription schema) {
+    if (DIGIT_CP_REPLACEMENT == UNMASKED_CHAR) {
+      return new DecimalIdentity();
+    } else {
+      return new DecimalRedactConverter();
+    }
+  }
+
+  @Override
+  protected DataMask buildDoubleMask(TypeDescription schema) {
+    if (DIGIT_CP_REPLACEMENT == UNMASKED_CHAR) {
+      return new DoubleIdentity();
+    } else {
+      return new DoubleRedactConverter();
+    }
+  }
+
+  @Override
+  protected DataMask buildStringMask(TypeDescription schema) {
+    return new StringConverter();
+  }
+
+  @Override
+  protected DataMask buildDateMask(TypeDescription schema) {
+    if (maskDate) {
+      return new DateRedactConverter();
+    } else {
+      return new LongIdentity();
+    }
+  }
+
+  @Override
+  protected DataMask buildTimestampMask(TypeDescription schema) {
+    if (maskTimestamp) {
+      return new TimestampRedactConverter();
+    } else {
+      return new TimestampIdentity();
+    }
+  }
+
+  @Override
+  protected DataMask buildBinaryMask(TypeDescription schema) {
+    return new NullifyMask();
+  }
+
+  class LongRedactConverter implements DataMask {
+    final long mask;
+
+    LongRedactConverter(TypeDescription.Category category) {
+      switch (category) {
+        case BYTE:
+          mask = 0xff;
+          break;
+        case SHORT:
+          mask = 0xffff;
+          break;
+        case INT:
+          mask = 0xffff_ffff;
+          break;
+        default:
+        case LONG:
+          mask = -1;
+          break;
+      }
+    }
+
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      LongColumnVector target = (LongColumnVector) masked;
+      LongColumnVector source = (LongColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      if (original.isRepeating) {
+        target.vector[0] = maskLong(source.vector[0]) & mask;
+        target.isNull[0] = source.isNull[0];
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.vector[r] = maskLong(source.vector[r]) & mask;
+          target.isNull[r] = source.isNull[r];
+        }
+      }
+    }
+  }
+
+  class BooleanRedactConverter implements DataMask {
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      LongColumnVector target = (LongColumnVector) masked;
+      LongColumnVector source = (LongColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      if (original.isRepeating) {
+        target.vector[0] = DIGIT_REPLACEMENT == 0 ? 0 : 1;
+        target.isNull[0] = source.isNull[0];
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.vector[r] = DIGIT_REPLACEMENT == 0 ? 0 : 1;
+          target.isNull[r] = source.isNull[r];
+        }
+      }
+    }
+  }
+
+  class DoubleRedactConverter implements DataMask {
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      DoubleColumnVector target = (DoubleColumnVector) masked;
+      DoubleColumnVector source = (DoubleColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      if (original.isRepeating) {
+        target.vector[0] = maskDouble(source.vector[0]);
+        target.isNull[0] = source.isNull[0];
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.vector[r] = maskDouble(source.vector[r]);
+          target.isNull[r] = source.isNull[r];
+        }
+      }
+    }
+  }
+
+  class StringConverter implements DataMask {
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      BytesColumnVector target = (BytesColumnVector) masked;
+      BytesColumnVector source = (BytesColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      if (original.isRepeating) {
+        target.isNull[0] = source.isNull[0];
+        if (target.noNulls || !target.isNull[0]) {
+          maskString(source, 0, target);
+        }
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.isNull[r] = source.isNull[r];
+          if (target.noNulls || !target.isNull[r]) {
+            maskString(source, r, target);
+          }
+        }
+      }
+    }
+  }
+
+  class DecimalRedactConverter implements DataMask {
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      DecimalColumnVector target = (DecimalColumnVector) masked;
+      DecimalColumnVector source = (DecimalColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      target.scale = source.scale;
+      target.precision = source.precision;
+      if (original.isRepeating) {
+        target.isNull[0] = source.isNull[0];
+        if (target.noNulls || !target.isNull[0]) {
+          target.vector[0].set(maskDecimal(source.vector[0]));
+        }
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.isNull[r] = source.isNull[r];
+          if (target.noNulls || !target.isNull[r]) {
+            target.vector[r].set(source.vector[r]);
+          }
+        }
+      }
+    }
+  }
+
+  class TimestampRedactConverter implements DataMask {
+
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      TimestampColumnVector target = (TimestampColumnVector) masked;
+      TimestampColumnVector source = (TimestampColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      if (original.isRepeating) {
+        target.isNull[0] = source.isNull[0];
+        if (target.noNulls || !target.isNull[0]) {
+          target.time[0] = maskTime(source.time[0]);
+          target.nanos[0] = 0;
+        }
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.isNull[r] = source.isNull[r];
+          if (target.noNulls || !target.isNull[r]) {
+            target.time[r] = maskTime(source.time[r]);
+            target.nanos[r] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  class DateRedactConverter implements DataMask {
+
+    @Override
+    public void maskData(ColumnVector original, ColumnVector masked, int start,
+                         int length) {
+      LongColumnVector target = (LongColumnVector) masked;
+      LongColumnVector source = (LongColumnVector) original;
+      target.noNulls = original.noNulls;
+      target.isRepeating = original.isRepeating;
+      if (original.isRepeating) {
+        target.isNull[0] = source.isNull[0];
+        if (target.noNulls || !target.isNull[0]) {
+          target.vector[0] = maskDate((int) source.vector[0]);
+        }
+      } else {
+        for(int r = start; r < start + length; ++r) {
+          target.isNull[r] = source.isNull[r];
+          if (target.noNulls || !target.isNull[r]) {
+            target.vector[r] = maskDate((int) source.vector[r]);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Get the next code point from the ByteBuffer. Moves the position in the
+   * ByteBuffer forward to the next code point.
+   * @param param the source of bytes
+   * @param defaultValue if there are no bytes left, use this value
+   * @return the code point that was found at the front of the buffer.
+   */
+  static int getNextCodepoint(ByteBuffer param, int defaultValue) {
+    if (param.remaining() == 0) {
+      return defaultValue;
+    } else {
+      return Text.bytesToCodePoint(param);
+    }
+  }
+
+  /**
+   * Get the replacement digit. This routine supports non-ASCII values for the
+   * replacement. For example, if the user gives one of "7", "७", "〧" or "፯"
+   * the value is 7.
+   * @param digitCodePoint the code point that is replacing digits
+   * @return the number from 0 to 9 to use as the numeric replacement
+   */
+  static int getReplacementDigit(int digitCodePoint) {
+    int dig = Character.getNumericValue(digitCodePoint);
+    if (dig >= 0 && dig <= 9) {
+      return dig;
+    } else {
+      return DEFAULT_NUMBER_DIGIT;
+    }
+  }
+
+  static int getDateParam(String[] dateParams, int posn,
+                          int myDefault, int max) {
+    if (dateParams != null && posn < dateParams.length) {
+      if (dateParams[posn].codePointAt(0) == UNMASKED_CHAR) {
+        return UNMASKED_DATE;
+      } else {
+        int result = Integer.parseInt(dateParams[posn]);
+        if (result >= -1 && result <= max) {
+          return result;
+        } else {
+          throw new IllegalArgumentException("Invalid date parameter " + posn +
+              " of " + dateParams[posn] + " greater than " + max);
+        }
+      }
+    } else {
+      return myDefault;
+    }
+  }
+
+  /**
+   * Replace each digit in value with DIGIT_REPLACEMENT scaled to the matching
+   * number of digits.
+   * @param value the number to mask
+   * @return the masked value
+   */
+  public long maskLong(long value) {
+    long base;
+    if (DIGIT_REPLACEMENT == 0) {
+      return 0;
+    } else if (value >= 0) {
+      base = 1;
+    } else {
+      base = -1;
+      // make sure Long.MIN_VALUE doesn't overflow
+      if (value == Long.MIN_VALUE) {
+        value = Long.MAX_VALUE;
+      } else {
+        value = -value;
+      }
+    }
+    if (value < 100_000_000L) {
+      if (value < 10_000L) {
+        if (value < 100L) {
+          if (value < 10L) {
+            base *= 1;
+          } else {
+            base *= 11;
+          }
+        } else if (value < 1_000L) {
+          base *= 111;
+        } else {
+          base *= 1_111;
+        }
+      } else if (value < 1_000_000L) {
+        if (value < 100_000L) {
+          base *= 11_111;
+        } else {
+          base *= 111_111;
+        }
+      } else if (value < 10_000_000L) {
+        base *= 1_111_111;
+      } else {
+        base *= 11_111_111;
+      }
+    } else if (value < 10_000_000_000_000_000L) {
+      if (value < 1_000_000_000_000L) {
+        if (value < 10_000_000_000L) {
+          if (value < 1_000_000_000L) {
+            base *= 111_111_111;
+          } else {
+            base *= 1_111_111_111;
+          }
+        } else if (value < 100_000_000_000L) {
+          base *= 11_111_111_111L;
+        } else {
+          base *= 111_111_111_111L;
+        }
+      } else if (value < 100_000_000_000_000L) {
+        if (value < 10_000_000_000_000L) {
+          base *= 1_111_111_111_111L;
+        } else {
+          base *= 11_111_111_111_111L;
+        }
+      } else if (value < 1_000_000_000_000_000L) {
+        base *= 111_111_111_111_111L;
+      } else {
+        base *= 1_111_111_111_111_111L;
+      }
+    } else if (value < 100_000_000_000_000_000L) {
+      base *= 11_111_111_111_111_111L;
+    // If the digit is 9, it would overflow at 19 digits, so use 18.
+    } else if (value < 1_000_000_000_000_000_000L || DIGIT_REPLACEMENT == 9) {
+      base *= 111_111_111_111_111_111L;
+    } else {
+      base *= 1_111_111_111_111_111_111L;
+    }
+    return DIGIT_REPLACEMENT * base;
+  }
+
+  private static final double[] DOUBLE_POWER_10 = new double[]{
+      1e-308, 1e-307, 1e-306, 1e-305, 1e-304, 1e-303, 1e-302, 1e-301, 1e-300,
+      1e-299, 1e-298, 1e-297, 1e-296, 1e-295, 1e-294, 1e-293, 1e-292, 1e-291,
+      1e-290, 1e-289, 1e-288, 1e-287, 1e-286, 1e-285, 1e-284, 1e-283, 1e-282,
+      1e-281, 1e-280, 1e-279, 1e-278, 1e-277, 1e-276, 1e-275, 1e-274, 1e-273,
+      1e-272, 1e-271, 1e-270, 1e-269, 1e-268, 1e-267, 1e-266, 1e-265, 1e-264,
+      1e-263, 1e-262, 1e-261, 1e-260, 1e-259, 1e-258, 1e-257, 1e-256, 1e-255,
+      1e-254, 1e-253, 1e-252, 1e-251, 1e-250, 1e-249, 1e-248, 1e-247, 1e-246,
+      1e-245, 1e-244, 1e-243, 1e-242, 1e-241, 1e-240, 1e-239, 1e-238, 1e-237,
+      1e-236, 1e-235, 1e-234, 1e-233, 1e-232, 1e-231, 1e-230, 1e-229, 1e-228,
+      1e-227, 1e-226, 1e-225, 1e-224, 1e-223, 1e-222, 1e-221, 1e-220, 1e-219,
+      1e-218, 1e-217, 1e-216, 1e-215, 1e-214, 1e-213, 1e-212, 1e-211, 1e-210,
+      1e-209, 1e-208, 1e-207, 1e-206, 1e-205, 1e-204, 1e-203, 1e-202, 1e-201,
+      1e-200, 1e-199, 1e-198, 1e-197, 1e-196, 1e-195, 1e-194, 1e-193, 1e-192,
+      1e-191, 1e-190, 1e-189, 1e-188, 1e-187, 1e-186, 1e-185, 1e-184, 1e-183,
+      1e-182, 1e-181, 1e-180, 1e-179, 1e-178, 1e-177, 1e-176, 1e-175, 1e-174,
+      1e-173, 1e-172, 1e-171, 1e-170, 1e-169, 1e-168, 1e-167, 1e-166, 1e-165,
+      1e-164, 1e-163, 1e-162, 1e-161, 1e-160, 1e-159, 1e-158, 1e-157, 1e-156,
+      1e-155, 1e-154, 1e-153, 1e-152, 1e-151, 1e-150, 1e-149, 1e-148, 1e-147,
+      1e-146, 1e-145, 1e-144, 1e-143, 1e-142, 1e-141, 1e-140, 1e-139, 1e-138,
+      1e-137, 1e-136, 1e-135, 1e-134, 1e-133, 1e-132, 1e-131, 1e-130, 1e-129,
+      1e-128, 1e-127, 1e-126, 1e-125, 1e-124, 1e-123, 1e-122, 1e-121, 1e-120,
+      1e-119, 1e-118, 1e-117, 1e-116, 1e-115, 1e-114, 1e-113, 1e-112, 1e-111,
+      1e-110, 1e-109, 1e-108, 1e-107, 1e-106, 1e-105, 1e-104, 1e-103, 1e-102,
+      1e-101, 1e-100, 1e-99, 1e-98, 1e-97, 1e-96, 1e-95, 1e-94, 1e-93,
+      1e-92, 1e-91, 1e-90, 1e-89, 1e-88, 1e-87, 1e-86, 1e-85, 1e-84,
+      1e-83, 1e-82, 1e-81, 1e-80, 1e-79, 1e-78, 1e-77, 1e-76, 1e-75,
+      1e-74, 1e-73, 1e-72, 1e-71, 1e-70, 1e-69, 1e-68, 1e-67, 1e-66,
+      1e-65, 1e-64, 1e-63, 1e-62, 1e-61, 1e-60, 1e-59, 1e-58, 1e-57,
+      1e-56, 1e-55, 1e-54, 1e-53, 1e-52, 1e-51, 1e-50, 1e-49, 1e-48,
+      1e-47, 1e-46, 1e-45, 1e-44, 1e-43, 1e-42, 1e-41, 1e-40, 1e-39,
+      1e-38, 1e-37, 1e-36, 1e-35, 1e-34, 1e-33, 1e-32, 1e-31, 1e-30,
+      1e-29, 1e-28, 1e-27, 1e-26, 1e-25, 1e-24, 1e-23, 1e-22, 1e-21,
+      1e-20, 1e-19, 1e-18, 1e-17, 1e-16, 1e-15, 1e-14, 1e-13, 1e-12,
+      1e-11, 1e-10, 1e-9, 1e-8, 1e-7, 1e-6, 1e-5, 1e-4, 1e-3,
+      1e-2, 1e-1, 1e0, 1e1, 1e2, 1e3, 1e4, 1e5, 1e6,
+      1e7, 1e8, 1e9, 1e10, 1e11, 1e12, 1e13, 1e14, 1e15,
+      1e16, 1e17, 1e18, 1e19, 1e20, 1e21, 1e22, 1e23, 1e24,
+      1e25, 1e26, 1e27, 1e28, 1e29, 1e30, 1e31, 1e32, 1e33,
+      1e34, 1e35, 1e36, 1e37, 1e38, 1e39, 1e40, 1e41, 1e42,
+      1e43, 1e44, 1e45, 1e46, 1e47, 1e48, 1e49, 1e50, 1e51,
+      1e52, 1e53, 1e54, 1e55, 1e56, 1e57, 1e58, 1e59, 1e60,
+      1e61, 1e62, 1e63, 1e64, 1e65, 1e66, 1e67, 1e68, 1e69,
+      1e70, 1e71, 1e72, 1e73, 1e74, 1e75, 1e76, 1e77, 1e78,
+      1e79, 1e80, 1e81, 1e82, 1e83, 1e84, 1e85, 1e86, 1e87,
+      1e88, 1e89, 1e90, 1e91, 1e92, 1e93, 1e94, 1e95, 1e96,
+      1e97, 1e98, 1e99, 1e100, 1e101, 1e102, 1e103, 1e104, 1e105,
+      1e106, 1e107, 1e108, 1e109, 1e110, 1e111, 1e112, 1e113, 1e114,
+      1e115, 1e116, 1e117, 1e118, 1e119, 1e120, 1e121, 1e122, 1e123,
+      1e124, 1e125, 1e126, 1e127, 1e128, 1e129, 1e130, 1e131, 1e132,
+      1e133, 1e134, 1e135, 1e136, 1e137, 1e138, 1e139, 1e140, 1e141,
+      1e142, 1e143, 1e144, 1e145, 1e146, 1e147, 1e148, 1e149, 1e150,
+      1e151, 1e152, 1e153, 1e154, 1e155, 1e156, 1e157, 1e158, 1e159,
+      1e160, 1e161, 1e162, 1e163, 1e164, 1e165, 1e166, 1e167, 1e168,
+      1e169, 1e170, 1e171, 1e172, 1e173, 1e174, 1e175, 1e176, 1e177,
+      1e178, 1e179, 1e180, 1e181, 1e182, 1e183, 1e184, 1e185, 1e186,
+      1e187, 1e188, 1e189, 1e190, 1e191, 1e192, 1e193, 1e194, 1e195,
+      1e196, 1e197, 1e198, 1e199, 1e200, 1e201, 1e202, 1e203, 1e204,
+      1e205, 1e206, 1e207, 1e208, 1e209, 1e210, 1e211, 1e212, 1e213,
+      1e214, 1e215, 1e216, 1e217, 1e218, 1e219, 1e220, 1e221, 1e222,
+      1e223, 1e224, 1e225, 1e226, 1e227, 1e228, 1e229, 1e230, 1e231,
+      1e232, 1e233, 1e234, 1e235, 1e236, 1e237, 1e238, 1e239, 1e240,
+      1e241, 1e242, 1e243, 1e244, 1e245, 1e246, 1e247, 1e248, 1e249,
+      1e250, 1e251, 1e252, 1e253, 1e254, 1e255, 1e256, 1e257, 1e258,
+      1e259, 1e260, 1e261, 1e262, 1e263, 1e264, 1e265, 1e266, 1e267,
+      1e268, 1e269, 1e270, 1e271, 1e272, 1e273, 1e274, 1e275, 1e276,
+      1e277, 1e278, 1e279, 1e280, 1e281, 1e282, 1e283, 1e284, 1e285,
+      1e286, 1e287, 1e288, 1e289, 1e290, 1e291, 1e292, 1e293, 1e294,
+      1e295, 1e296, 1e297, 1e298, 1e299, 1e300, 1e301, 1e302, 1e303,
+      1e304, 1e305, 1e306, 1e307};
+
+  /**
+   * Replace each digit in value with digit.
+   * @param value the number to mask
+   * @return the
+   */
+  public double maskDouble(double value) {
+    double base;
+    // It seems better to mask 0 to 9.99999 rather than 9.99999e-308.
+    if (value == 0 || DIGIT_REPLACEMENT == 0) {
+      return DIGIT_REPLACEMENT * 1.11111;
+    } else if (value > 0) {
+      base = 1.11111;
+    } else {
+      base = -1.11111;
+      value = -value;
+    }
+    int posn = Arrays.binarySearch(DOUBLE_POWER_10, value);
+    if (posn < -DOUBLE_POWER_10.length - 2) {
+      posn = DOUBLE_POWER_10.length - 1;
+    } else if (posn == -1) {
+      posn = 0;
+    } else if (posn < 0) {
+      posn = -posn -2;
+    }
+    return DIGIT_REPLACEMENT * base * DOUBLE_POWER_10[posn];
+  }
+
+  private final Calendar scratch = Calendar.getInstance();
+
+  /**
+   * Given the requested masking parameters, redact the given time
+   * @param millis the original time
+   * @return the millis after it has been masked
+   */
+  long maskTime(long millis) {
+    scratch.setTimeInMillis(millis);
+    if (YEAR_REPLACEMENT != UNMASKED_DATE) {
+      scratch.set(Calendar.YEAR, YEAR_REPLACEMENT);
+    }
+    if (MONTH_REPLACEMENT != UNMASKED_DATE) {
+      scratch.set(Calendar.MONTH, MONTH_REPLACEMENT - 1);
+    }
+    if (DATE_REPLACEMENT != UNMASKED_DATE) {
+      scratch.set(Calendar.DATE, DATE_REPLACEMENT);
+    }
+    if (HOUR_REPLACEMENT != UNMASKED_DATE) {
+      if (HOUR_REPLACEMENT >= 12) {
+        scratch.set(Calendar.HOUR, HOUR_REPLACEMENT - 12);
+        scratch.set(Calendar.AM_PM, Calendar.PM);
+      } else {
+        scratch.set(Calendar.HOUR, HOUR_REPLACEMENT);
+        scratch.set(Calendar.AM_PM, Calendar.AM);
+      }
+    }
+    if (MINUTE_REPLACEMENT != UNMASKED_DATE) {
+      scratch.set(Calendar.MINUTE, MINUTE_REPLACEMENT);
+    }
+    if (SECOND_REPLACEMENT != UNMASKED_DATE) {
+      scratch.set(Calendar.SECOND, SECOND_REPLACEMENT);
+      scratch.set(Calendar.MILLISECOND, 0);
+    }
+    return scratch.getTimeInMillis();
+  }
+
+  private static final long MILLIS_PER_DAY = TimeUnit.DAYS.toMillis(1);
+
+  private final Calendar utcScratch =
+      Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+
+  /**
+   * Given a date as the number of days since epoch (1 Jan 1970),
+   * mask the date given the parameters.
+   * @param daysSinceEpoch the number of days after epoch
+   * @return the number of days after epoch when masked
+   */
+  int maskDate(int daysSinceEpoch) {
+    utcScratch.setTimeInMillis(daysSinceEpoch * MILLIS_PER_DAY);
+    if (YEAR_REPLACEMENT != UNMASKED_DATE) {
+      utcScratch.set(Calendar.YEAR, YEAR_REPLACEMENT);
+    }
+    if (MONTH_REPLACEMENT != UNMASKED_DATE) {
+      utcScratch.set(Calendar.MONTH, MONTH_REPLACEMENT - 1);
+    }
+    if (DATE_REPLACEMENT != UNMASKED_DATE) {
+      utcScratch.set(Calendar.DATE, DATE_REPLACEMENT);
+    }
+    return (int) (utcScratch.getTimeInMillis() / MILLIS_PER_DAY);
+  }
+
+  private static final Pattern DIGIT_PATTERN = Pattern.compile("[0-9]");
+
+  /**
+   * Mask a decimal.
+   * This is painfully slow because it converts to a string and then back to
+   * a decimal. Until HiveDecimalWritable gives us more access, this is
+   * the best tradeoff between developer time, functionality, and run time.
+   * @param source the value to mask
+   * @return the masked value.
+   */
+  HiveDecimalWritable maskDecimal(HiveDecimalWritable source) {
+    String str = DIGIT_PATTERN.matcher(source.toString()).
+        replaceAll(Integer.toString(DIGIT_REPLACEMENT));
+    return new HiveDecimalWritable(str);
+  }
+
+  /**
+   * Given a UTF code point, find the replacement codepoint
+   * @param codepoint a UTF character
+   * @return the replacement codepoint
+   */
+  int getReplacement(int codepoint) {
+    switch (Character.getType(codepoint)) {
+      case Character.UPPERCASE_LETTER:
+        return UPPPER_REPLACEMENT;
+      case Character.LOWERCASE_LETTER:
+        return LOWER_REPLACEMENT;
+      case Character.TITLECASE_LETTER:
+      case Character.MODIFIER_LETTER:
+      case Character.OTHER_LETTER:
+        return OTHER_LETTER_REPLACEMENT;
+      case Character.NON_SPACING_MARK:
+      case Character.ENCLOSING_MARK:
+      case Character.COMBINING_SPACING_MARK:
+        return MARK_REPLACEMENT;
+      case Character.DECIMAL_DIGIT_NUMBER:
+        return DIGIT_CP_REPLACEMENT;
+      case Character.LETTER_NUMBER:
+      case Character.OTHER_NUMBER:
+        return OTHER_NUMBER_REPLACEMENT;
+      case Character.SPACE_SEPARATOR:
+      case Character.LINE_SEPARATOR:
+      case Character.PARAGRAPH_SEPARATOR:
+        return SEPARATOR_REPLACEMENT;
+      case Character.MATH_SYMBOL:
+      case Character.CURRENCY_SYMBOL:
+      case Character.MODIFIER_SYMBOL:
+      case Character.OTHER_SYMBOL:
+        return SYMBOL_REPLACEMENT;
+      case Character.DASH_PUNCTUATION:
+      case Character.START_PUNCTUATION:
+      case Character.END_PUNCTUATION:
+      case Character.CONNECTOR_PUNCTUATION:
+      case Character.OTHER_PUNCTUATION:
+        return PUNCTUATION_REPLACEMENT;
+      default:
+        return OTHER_REPLACEMENT;
+    }
+  }
+
+  /**
+   * Get the number of bytes for each codepoint
+   * @param codepoint the codepoint to check
+   * @return the number of bytes
+   */
+  static int getCodepointLength(int codepoint) {
+    if (codepoint < 0) {
+      throw new IllegalArgumentException("Illegal codepoint " + codepoint);
+    } else if (codepoint < 0x80) {
+      return 1;
+    } else if (codepoint < 0x7ff) {
+      return 2;
+    } else if (codepoint < 0xffff) {
+      return 3;
+    } else if (codepoint < 0x10FFFF) {
+      return 4;
+    } else {
+      throw new IllegalArgumentException("Illegal codepoint " + codepoint);
+    }
+  }
+
+  /**
+   * Write the give codepoint to the buffer.
+   * @param codepoint the codepoint to write
+   * @param buffer the buffer to write into
+   * @param offset the first offset to use
+   * @param length the number of bytes that will be used
+   */
+  static void writeCodepoint(int codepoint, byte[] buffer, int offset,
+                             int length) {
+    switch (length) {
+      case 1:
+        buffer[offset] = (byte) codepoint;
+        break;
+      case 2:
+        buffer[offset] = (byte)(0xC0 | codepoint >> 6);
+        buffer[offset+1] = (byte)(0x80 | (codepoint & 0x3f));
+        break;
+      case 3:
+        buffer[offset] = (byte)(0xE0 | codepoint >> 12);
+        buffer[offset+1] = (byte)(0x80 | ((codepoint >> 6) & 0x3f));
+        buffer[offset+2] = (byte)(0x80 | (codepoint & 0x3f));
+        break;
+      case 4:
+        buffer[offset] = (byte)(0xF0 | codepoint >> 18);
+        buffer[offset+1] = (byte)(0x80 | ((codepoint >> 12) & 0x3f));
+        buffer[offset+2] = (byte)(0x80 | ((codepoint >> 6) & 0x3f));
+        buffer[offset+3] = (byte)(0x80 | (codepoint & 0x3f));
+        break;
+      default:
+        throw new IllegalArgumentException("Invalid length for codepoint " +
+            codepoint + " = " + length);
+    }
+  }
+
+  /**
+   * Mask a string by finding the character category of each character
+   * and replacing it with the matching literal.
+   * @param source the source column vector
+   * @param row the value index
+   * @param target the target column vector
+   */
+  void maskString(BytesColumnVector source, int row, BytesColumnVector target) {
+    int expectedBytes = source.length[row];
+    ByteBuffer sourceBytes = ByteBuffer.wrap(source.vector[row],
+        source.start[row], source.length[row]);
+    // ensure we have enough space, if the masked data is the same size
+    target.ensureValPreallocated(expectedBytes);
+    byte[] outputBuffer = target.getValPreallocatedBytes();
+    int outputOffset = target.getValPreallocatedStart();
+    int outputStart = outputOffset;
+    while (sourceBytes.remaining() > 0) {
+      int cp = Text.bytesToCodePoint(sourceBytes);
+
+      // Find the replacement for the current character.
+      int replacement = getReplacement(cp);
+      if (replacement == UNMASKED_CHAR) {
+        replacement = cp;
+      }
+      int len = getCodepointLength(replacement);
+
+      // If the translation will overflow the buffer, we need to resize.
+      // This will only happen when the masked size is larger than the original.
+      if (len + outputOffset > outputBuffer.length) {
+        // Revise estimate how much we are going to need now. We are maximally
+        // pesamistic here so that we don't have to expand again for this value.
+        int currentOutputStart =  outputStart;
+        int currentOutputLength = outputOffset - currentOutputStart;
+        expectedBytes = currentOutputLength + len + sourceBytes.remaining() * 4;
+
+        // Expand the buffer to fit the new estimate
+        target.ensureValPreallocated(expectedBytes);
+
+        // Copy over the bytes we've already written for this value and move
+        // the pointers to the new output buffer.
+        byte[] oldBuffer = outputBuffer;
+        outputBuffer = target.getValPreallocatedBytes();
+        outputOffset = target.getValPreallocatedStart();
+        outputStart = outputOffset;
+        System.arraycopy(oldBuffer, currentOutputStart, outputBuffer,
+            outputOffset, currentOutputLength);
+        outputOffset += currentOutputLength;
+      }
+
+      // finally copy the bytes
+      writeCodepoint(replacement, outputBuffer, outputOffset, len);
+      outputOffset += len;
+    }
+    target.setValPreallocated(row, outputOffset - outputStart);
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/StructIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/StructIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/StructIdentity.java
new file mode 100644
index 0000000..92cfb4e
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/StructIdentity.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * A data mask for struct types that applies the given masks to its
+ * children, but doesn't mask at this level.
+ */
+public class StructIdentity implements DataMask {
+  private final DataMask[] children;
+
+  StructIdentity(DataMask[] children) {
+    this.children = children;
+  }
+
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    StructColumnVector source = (StructColumnVector) original;
+    StructColumnVector target = (StructColumnVector) masked;
+    target.isRepeating = source.isRepeating;
+    target.noNulls = source.noNulls;
+    if (source.isRepeating) {
+      target.isNull[0] = source.isNull[0];
+      if (source.noNulls || !source.isNull[0]) {
+        for (int c = 0; c < children.length; ++c) {
+          children[c].maskData(source.fields[c], target.fields[c], 0, 1);
+        }
+      }
+    } else if (source.noNulls) {
+      for (int c = 0; c < children.length; ++c) {
+        children[c].maskData(source.fields[c], target.fields[c], start, length);
+      }
+    } else {
+      // process the children in runs of non-null values
+      int batchStart = start;
+      while (batchStart < start + length) {
+        int r = batchStart;
+        while (r < start + length && !source.isNull[r]) {
+          r += 1;
+        }
+        if (r != batchStart) {
+          for(int c=0; c < children.length; ++c) {
+            children[c].maskData(source.fields[c], target.fields[c],
+                batchStart, r - batchStart);
+          }
+        }
+        batchStart = r;
+        while (batchStart < start + length && source.isNull[batchStart]) {
+          batchStart += 1;
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/TimestampIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/TimestampIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/TimestampIdentity.java
new file mode 100644
index 0000000..b0b2684
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/TimestampIdentity.java
@@ -0,0 +1,52 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.DataMask;
+
+class TimestampIdentity implements DataMask {
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    TimestampColumnVector target = (TimestampColumnVector) masked;
+    TimestampColumnVector source = (TimestampColumnVector) original;
+    target.noNulls = source.noNulls;
+    target.isRepeating = source.isRepeating;
+    if (original.isRepeating) {
+      target.time[0] = source.time[0];
+      target.nanos[0] = source.nanos[0];
+      target.isNull[0] = source.isNull[0];
+    } else if (source.noNulls) {
+      for(int r = start; r < start + length; ++r) {
+        target.time[r] = source.time[r];
+        target.nanos[r] = source.nanos[r];
+      }
+    } else {
+      for(int r = start; r < start + length; ++r) {
+        target.time[r] = source.time[r];
+        target.nanos[r] = source.nanos[r];
+        target.isNull[r] = source.isNull[r];
+      }
+    }
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/java/org/apache/orc/impl/mask/UnionIdentity.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/mask/UnionIdentity.java b/java/core/src/java/org/apache/orc/impl/mask/UnionIdentity.java
new file mode 100644
index 0000000..27ae46c
--- /dev/null
+++ b/java/core/src/java/org/apache/orc/impl/mask/UnionIdentity.java
@@ -0,0 +1,66 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.orc.DataMask;
+
+/**
+ * A data mask for union types that applies the given masks to its
+ * children, but doesn't mask at this level.
+ */
+public class UnionIdentity implements DataMask {
+  private final DataMask[] children;
+
+  UnionIdentity(DataMask[] children) {
+    this.children = children;
+  }
+
+  @Override
+  public void maskData(ColumnVector original, ColumnVector masked, int start,
+                       int length) {
+    UnionColumnVector source = (UnionColumnVector) original;
+    UnionColumnVector target = (UnionColumnVector) masked;
+    target.isRepeating = source.isRepeating;
+    target.noNulls = source.noNulls;
+    if (source.isRepeating) {
+      target.isNull[0] = source.isNull[0];
+      if (source.noNulls || !source.isNull[0]) {
+        int tag = source.tags[0];
+        target.tags[0] = tag;
+        children[tag].maskData(source.fields[tag], target.fields[tag], 0, 1);
+      }
+    } else if (source.noNulls) {
+      for (int r = start; r < start + length; ++r) {
+        int tag = source.tags[r];
+        target.tags[r] = tag;
+        children[tag].maskData(source.fields[tag], target.fields[tag], r, 1);
+      }
+    } else {
+      for(int r= start; r < start + length; ++r) {
+        target.isNull[r] = source.isNull[r];
+        if (!source.isNull[r]) {
+          int tag = source.tags[r];
+          target.tags[r] = tag;
+          children[tag].maskData(source.fields[tag], target.fields[tag], r, 1);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/resources/META-INF/services/org.apache.orc.DataMask$Provider
----------------------------------------------------------------------
diff --git a/java/core/src/resources/META-INF/services/org.apache.orc.DataMask$Provider b/java/core/src/resources/META-INF/services/org.apache.orc.DataMask$Provider
new file mode 100644
index 0000000..6daf1b0
--- /dev/null
+++ b/java/core/src/resources/META-INF/services/org.apache.orc.DataMask$Provider
@@ -0,0 +1,15 @@
+# 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.
+org.apache.orc.impl.mask.MaskProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/orc/blob/d5a6c49b/java/core/src/test/org/apache/orc/impl/mask/TestDataMask.java
----------------------------------------------------------------------
diff --git a/java/core/src/test/org/apache/orc/impl/mask/TestDataMask.java b/java/core/src/test/org/apache/orc/impl/mask/TestDataMask.java
new file mode 100644
index 0000000..bbf313c
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/impl/mask/TestDataMask.java
@@ -0,0 +1,177 @@
+/*
+ * 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.orc.impl.mask;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.orc.DataMask;
+import org.apache.orc.TypeDescription;
+import org.junit.Test;
+
+import java.nio.charset.StandardCharsets;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDataMask {
+
+  @Test
+  public void testNullFactory() throws Exception {
+    TypeDescription schema = TypeDescription.fromString("struct<x:int>");
+    // take the first column's type
+    DataMask mask = DataMask.Standard.NULLIFY.build(schema.findSubtype(1));
+    assertEquals(NullifyMask.class.toString(), mask.getClass().toString());
+    LongColumnVector cv = (LongColumnVector) schema.createRowBatch().cols[0];
+    LongColumnVector masked = (LongColumnVector) schema.createRowBatch().cols[0];
+    cv.vector[0] = 10;
+    cv.vector[1] = 20;
+    mask.maskData(cv, masked, 0, 2);
+    assertEquals(true, masked.isRepeating);
+    assertEquals(false, masked.noNulls);
+    assertEquals(true, masked.isNull[0]);
+  }
+
+  @Test
+  public void testRedactFactory() throws Exception {
+    TypeDescription schema =
+        TypeDescription.fromString("struct<s:struct<x:int,y:string>>");
+    DataMask mask = DataMask.Standard.REDACT.build(schema.findSubtype(1));
+    assertEquals(StructIdentity.class.toString(), mask.getClass().toString());
+    StructColumnVector cv = (StructColumnVector)schema.createRowBatch().cols[0];
+    StructColumnVector masked = (StructColumnVector)schema.createRowBatch().cols[0];
+    LongColumnVector x = (LongColumnVector) cv.fields[0];
+    BytesColumnVector y = (BytesColumnVector) cv.fields[1];
+    x.vector[0] = 123;
+    y.setVal(0, "Owen".getBytes(StandardCharsets.UTF_8));
+    x.vector[1] = 456789;
+    y.setVal(1, "ORC".getBytes(StandardCharsets.UTF_8));
+    mask.maskData(cv, masked, 0, 2);
+    x = (LongColumnVector) masked.fields[0];
+    y = (BytesColumnVector) masked.fields[1];
+    assertEquals(999, x.vector[0]);
+    assertEquals(999999, x.vector[1]);
+    assertEquals("Xxxx", y.toString(0));
+    assertEquals("XXX", y.toString(1));
+  }
+
+  @Test
+  public void testIdentityRedact() throws Exception {
+    TypeDescription schema =
+        TypeDescription.fromString("struct<s:struct<a:decimal(18,6),b:double," +
+            "c:array<int>,d:map<timestamp,date>,e:uniontype<int,binary>,f:string>>");
+    DataMask nullify = DataMask.Standard.NULLIFY.build(schema.findSubtype(1));
+    // create a redact mask that passes everything though
+    DataMask identity = DataMask.Standard.REDACT.build(schema.findSubtype(1),
+        "__________", "_ _ _ _ _ _");
+
+    // allow easier access to fields
+    StructColumnVector cv = (StructColumnVector)schema.createRowBatch().cols[0];
+    StructColumnVector masked = (StructColumnVector)schema.createRowBatch().cols[0];
+    DecimalColumnVector a = (DecimalColumnVector) cv.fields[0];
+    DoubleColumnVector b = (DoubleColumnVector) cv.fields[1];
+    ListColumnVector c = (ListColumnVector) cv.fields[2];
+    LongColumnVector ce = (LongColumnVector) c.child;
+    MapColumnVector d = (MapColumnVector) cv.fields[3];
+    TimestampColumnVector dk = (TimestampColumnVector) d.keys;
+    LongColumnVector dv = (LongColumnVector) d.values;
+    UnionColumnVector e = (UnionColumnVector) cv.fields[4];
+    LongColumnVector e1 = (LongColumnVector) e.fields[0];
+    BytesColumnVector e2 = (BytesColumnVector) e.fields[1];
+    BytesColumnVector f = (BytesColumnVector) cv.fields[5];
+
+    // set up the input data
+    for(int i=0; i < 3; ++i) {
+      a.set(i, new HiveDecimalWritable((i + 1) + "." + (i + 1)));
+      b.vector[i] = 1.25 * (i + 1);
+      // layout c normally
+      c.offsets[i] = i == 0 ? 0 : c.offsets[i-1] + c.lengths[i-1];
+      c.lengths[i] = 2 * i;
+      // layout d backward
+      d.offsets[i] = 2 * (2 - i);
+      d.lengths[i] = 2;
+      e.tags[i] = i % 2;
+      e1.vector[i] = i * 10;
+      f.setVal(i, Integer.toHexString(0x123 * i).getBytes(StandardCharsets.UTF_8));
+    }
+    e2.setVal(1, "Foobar".getBytes(StandardCharsets.UTF_8));
+    for(int i=0; i < 6; ++i) {
+      ce.vector[i] = i;
+      dk.time[i] = 1111 * i;
+      dk.nanos[i] = 0;
+      dv.vector[i] = i * 11;
+    }
+
+    // send it through the nullify mask
+    nullify.maskData(cv, masked, 0, 3);
+    assertEquals(false, masked.noNulls);
+    assertEquals(true, masked.isRepeating);
+    assertEquals(true, masked.isNull[0]);
+
+    // send it through our identity mask
+    identity.maskData(cv, masked, 0 , 3);
+    assertEquals(true, masked.noNulls);
+    assertEquals(false, masked.isRepeating);
+
+    // point accessors to masked values
+    a = (DecimalColumnVector) masked.fields[0];
+    b = (DoubleColumnVector) masked.fields[1];
+    c = (ListColumnVector) masked.fields[2];
+    ce = (LongColumnVector) c.child;
+    d = (MapColumnVector) masked.fields[3];
+    dk = (TimestampColumnVector) d.keys;
+    dv = (LongColumnVector) d.values;
+    e = (UnionColumnVector) masked.fields[4];
+    e1 = (LongColumnVector) e.fields[0];
+    e2 = (BytesColumnVector) e.fields[1];
+    f = (BytesColumnVector) masked.fields[5];
+
+    // check the outputs
+    for(int i=0; i < 3; ++i) {
+      assertEquals("iter " + i, (i + 1) + "." + (i + 1), a.vector[i].toString());
+      assertEquals("iter " + i, 1.25 * (i + 1), b.vector[i], 0.0001);
+      assertEquals("iter " + i, i == 0 ? 0 : c.offsets[i-1] + c.lengths[i-1], c.offsets[i]);
+      assertEquals("iter " + i, 2 * i, c.lengths[i]);
+      assertEquals("iter " + i, i == 0 ? 4 : d.offsets[i-1] - d.lengths[i], d.offsets[i]);
+      assertEquals("iter " + i, 2, d.lengths[i]);
+      assertEquals("iter " + i, i % 2, e.tags[i]);
+      assertEquals("iter " + i, Integer.toHexString(0x123 * i), f.toString(i));
+    }
+    // check the subvalues for the list and map
+    for(int i=0; i < 6; ++i) {
+      assertEquals("iter " + i, i, ce.vector[i]);
+      assertEquals("iter " + i, i * 1111, dk.time[i]);
+      assertEquals("iter " + i, i * 11, dv.vector[i]);
+    }
+    assertEquals(0, e1.vector[0]);
+    assertEquals(20, e1.vector[2]);
+    // the redact mask always replaces binary with null
+    assertEquals(false, e2.noNulls);
+    assertEquals(true, e2.isRepeating);
+    assertEquals(true, e2.isNull[0]);
+  }
+
+}