You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by rs...@apache.org on 2021/10/06 09:49:16 UTC

[avro] branch master updated: AVRO-2863: Create an avro-android artifact (#1347)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new da39228  AVRO-2863: Create an avro-android artifact (#1347)
da39228 is described below

commit da39228b606c9260652aaa583f460cb74997b197
Author: RyanSkraba <ry...@skraba.com>
AuthorDate: Wed Oct 6 11:49:08 2021 +0200

    AVRO-2863: Create an avro-android artifact (#1347)
    
    * AVRO-2863: Create a shaded jar for Android compatibility
    
    * AVRO-2863: Use MapEntry from outside reflect package
    
    * AVRO-2863: Override ClassValue in the android implmementation
    
    * AVRO-3862: Override ThreadLocal.withInitial in the android implementation
    
    * AVRO-2863: Reorder data members before constructors
    
    * AVRO-2863: Remove unwanted .class file
---
 lang/java/android/pom.xml                          | 114 +++++++++++++++++++++
 .../apache/avro/util/internal/ClassValueCache.java |  46 +++++++++
 .../avro/util/internal/ThreadLocalWithInitial.java |  38 +++++++
 .../avro/util/internal/TestClassValueCache.java    |  37 +++++++
 .../main/java/org/apache/avro/JsonProperties.java  |   2 +-
 .../avro/src/main/java/org/apache/avro/Schema.java |   9 +-
 .../apache/avro/generic/GenericDatumReader.java    |   5 +-
 .../main/java/org/apache/avro/io/BinaryData.java   |   5 +-
 .../apache/avro/message/BinaryMessageDecoder.java  |   5 +-
 .../org/apache/avro/message/MessageDecoder.java    |  10 +-
 .../org/apache/avro/message/RawMessageEncoder.java |   4 +-
 .../java/org/apache/avro/reflect/MapEntry.java     |   2 +
 .../apache/avro/reflect/ReflectDatumWriter.java    |   1 +
 .../org/apache/avro/specific/SpecificData.java     |  66 ++++++------
 .../apache/avro/{reflect => util}/MapEntry.java    |  15 +--
 .../apache/avro/util/internal/ClassValueCache.java |  47 +++++++++
 .../avro/util/internal/ThreadLocalWithInitial.java |  32 ++++++
 .../avro/util/internal/TestClassValueCache.java    |  37 +++++++
 lang/java/pom.xml                                  |   1 +
 19 files changed, 417 insertions(+), 59 deletions(-)

diff --git a/lang/java/android/pom.xml b/lang/java/android/pom.xml
new file mode 100644
index 0000000..e43bd5a
--- /dev/null
+++ b/lang/java/android/pom.xml
@@ -0,0 +1,114 @@
+<?xml version="1.0" encoding="UTF-8"?><!--
+   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
+
+       https://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.
+-->
+<project
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <artifactId>avro-parent</artifactId>
+    <groupId>org.apache.avro</groupId>
+    <version>1.11.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>avro-android</artifactId>
+
+  <name>Apache Avro Android Compatibility</name>
+  <url>https://avro.apache.org</url>
+  <description>A subset of the Avro core utilities for the Android platform</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <main.basedir>${project.parent.parent.basedir}</main.basedir>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <!-- primary artifact is shaded -->
+        <executions>
+          <execution>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <createDependencyReducedPom>true</createDependencyReducedPom>
+              <artifactSet>
+                <includes>
+                  <include>org.apache.avro:*</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>org.apache.avro:avro</artifact>
+                  <includes>
+                    <include>org/apache/avro/**</include>
+                  </includes>
+                  <excludes>
+                    <exclude>org/apache/avro/avro/util/internal/ClassValueUtil**</exclude>
+                    <exclude>org/apache/avro/avro/util/internal/ThreadLocalWithInitial**</exclude>
+                    <exclude>org/apache/avro/reflect/**</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-compress</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.tukaani</groupId>
+      <artifactId>xz</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>com.github.luben</groupId>
+      <artifactId>zstd-jni</artifactId>
+      <optional>true</optional>
+    </dependency>
+  </dependencies>
+
+</project>
diff --git a/lang/java/android/src/main/java/org/apache/avro/util/internal/ClassValueCache.java b/lang/java/android/src/main/java/org/apache/avro/util/internal/ClassValueCache.java
new file mode 100644
index 0000000..7728455
--- /dev/null
+++ b/lang/java/android/src/main/java/org/apache/avro/util/internal/ClassValueCache.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://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.avro.util.internal;
+
+import java.util.function.Function;
+
+/**
+ * The Android environment doesn't support {@link ClassValue}. This utility
+ * bypasses its use in Avro to always recalculate the value without caching.
+ * <p>
+ * This may have a performance impact in Android.
+ *
+ * @param <R> Return type of the ClassValue
+ */
+public class ClassValueCache<R> implements Function<Class<?>, R> {
+
+  private final Function<Class<?>, R> ifAbsent;
+
+  /**
+   * @param ifAbsent The function that calculates the value to be used from the
+   *                 class instance.
+   */
+  public ClassValueCache(Function<Class<?>, R> ifAbsent) {
+    this.ifAbsent = ifAbsent;
+  }
+
+  @Override
+  public R apply(Class<?> c) {
+    return ifAbsent.apply(c);
+  }
+}
diff --git a/lang/java/android/src/main/java/org/apache/avro/util/internal/ThreadLocalWithInitial.java b/lang/java/android/src/main/java/org/apache/avro/util/internal/ThreadLocalWithInitial.java
new file mode 100644
index 0000000..cdba5ec
--- /dev/null
+++ b/lang/java/android/src/main/java/org/apache/avro/util/internal/ThreadLocalWithInitial.java
@@ -0,0 +1,38 @@
+/*
+ * 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
+ *
+ *     https://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.avro.util.internal;
+
+import java.util.function.Supplier;
+
+/**
+ * Wraps a {@link ThreadLocal#withInitial(Supplier)} so it can be overridden in
+ * an android environment, where this method is not available until API 26.
+ */
+public class ThreadLocalWithInitial {
+
+  /** Delegate a ThreadLocal instance with the supplier. */
+  @SuppressWarnings("AnonymousHasLambdaAlternative")
+  public static <T> ThreadLocal<T> of(Supplier<? extends T> supplier) {
+    return new ThreadLocal<T>() {
+      @Override
+      protected T initialValue() {
+        return supplier.get();
+      }
+    };
+  }
+}
diff --git a/lang/java/android/src/test/java/org/apache/avro/util/internal/TestClassValueCache.java b/lang/java/android/src/test/java/org/apache/avro/util/internal/TestClassValueCache.java
new file mode 100644
index 0000000..09f6e4b
--- /dev/null
+++ b/lang/java/android/src/test/java/org/apache/avro/util/internal/TestClassValueCache.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://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.avro.util.internal;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.*;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public class TestClassValueCache {
+  @Test
+  public void testBasic() {
+    ClassValueCache<String> cache = new ClassValueCache<>(Class::toString);
+
+    String fromCache = cache.apply(String.class);
+    assertThat(fromCache, is("class java.lang.String"));
+
+    // Unlike the core ClassValueUtil, this always creates a new instance
+    assertThat(cache.apply(String.class), not(sameInstance(fromCache)));
+  }
+
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java b/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java
index e8c33a4..b53bc6c 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java
@@ -33,7 +33,7 @@ import java.io.IOException;
 
 import org.apache.avro.util.internal.Accessor;
 import org.apache.avro.util.internal.Accessor.JsonPropertiesAccessor;
-import org.apache.avro.reflect.MapEntry;
+import org.apache.avro.util.MapEntry;
 import org.apache.avro.util.internal.JacksonUtils;
 
 import com.fasterxml.jackson.core.JsonGenerator;
diff --git a/lang/java/avro/src/main/java/org/apache/avro/Schema.java b/lang/java/avro/src/main/java/org/apache/avro/Schema.java
index 89d0b32..76ee2ee 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/Schema.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/Schema.java
@@ -49,6 +49,7 @@ import java.util.Set;
 import org.apache.avro.util.internal.Accessor;
 import org.apache.avro.util.internal.Accessor.FieldAccessor;
 import org.apache.avro.util.internal.JacksonUtils;
+import org.apache.avro.util.internal.ThreadLocalWithInitial;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -864,8 +865,8 @@ public abstract class Schema extends JsonProperties implements Serializable {
     }
   }
 
-  private static final ThreadLocal<Set> SEEN_EQUALS = ThreadLocal.withInitial(HashSet::new);
-  private static final ThreadLocal<Map> SEEN_HASHCODE = ThreadLocal.withInitial(IdentityHashMap::new);
+  private static final ThreadLocal<Set> SEEN_EQUALS = ThreadLocalWithInitial.of(HashSet::new);
+  private static final ThreadLocal<Map> SEEN_HASHCODE = ThreadLocalWithInitial.of(IdentityHashMap::new);
 
   @SuppressWarnings(value = "unchecked")
   private static class RecordSchema extends NamedSchema {
@@ -1551,7 +1552,7 @@ public abstract class Schema extends JsonProperties implements Serializable {
     }
   }
 
-  private static ThreadLocal<Boolean> validateNames = ThreadLocal.withInitial(() -> true);
+  private static ThreadLocal<Boolean> validateNames = ThreadLocalWithInitial.of(() -> true);
 
   private static String validateName(String name) {
     if (!validateNames.get())
@@ -1572,7 +1573,7 @@ public abstract class Schema extends JsonProperties implements Serializable {
     return name;
   }
 
-  private static final ThreadLocal<Boolean> VALIDATE_DEFAULTS = ThreadLocal.withInitial(() -> true);
+  private static final ThreadLocal<Boolean> VALIDATE_DEFAULTS = ThreadLocalWithInitial.of(() -> true);
 
   private static JsonNode validateDefault(String fieldName, Schema schema, JsonNode defaultValue) {
     if (VALIDATE_DEFAULTS.get() && (defaultValue != null) && !isValidDefault(schema, defaultValue)) { // invalid default
diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
index 83e5e80..e32db6e 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
@@ -38,6 +38,7 @@ import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.ResolvingDecoder;
 import org.apache.avro.util.Utf8;
 import org.apache.avro.util.WeakIdentityHashMap;
+import org.apache.avro.util.internal.ThreadLocalWithInitial;
 
 /** {@link DatumReader} for generic Java objects. */
 public class GenericDatumReader<D> implements DatumReader<D> {
@@ -105,8 +106,8 @@ public class GenericDatumReader<D> implements DatumReader<D> {
     creatorResolver = null;
   }
 
-  private static final ThreadLocal<Map<Schema, Map<Schema, ResolvingDecoder>>> RESOLVER_CACHE = ThreadLocal
-      .withInitial(WeakIdentityHashMap::new);
+  private static final ThreadLocal<Map<Schema, Map<Schema, ResolvingDecoder>>> RESOLVER_CACHE = ThreadLocalWithInitial
+      .of(WeakIdentityHashMap::new);
 
   /**
    * Gets a resolving decoder for use by this GenericDatumReader. Unstable API.
diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java
index b0fc87a..f925bcd 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java
@@ -23,6 +23,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.AvroRuntimeException;
 import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.util.internal.ThreadLocalWithInitial;
 
 /** Utilities for binary-encoded data. */
 public class BinaryData {
@@ -49,7 +50,7 @@ public class BinaryData {
     }
   } // no public ctor
 
-  private static final ThreadLocal<Decoders> DECODERS = ThreadLocal.withInitial(Decoders::new);
+  private static final ThreadLocal<Decoders> DECODERS = ThreadLocalWithInitial.of(Decoders::new);
 
   /**
    * Compare binary encoded data. If equal, return zero. If greater-than, return
@@ -204,7 +205,7 @@ public class BinaryData {
     }
   }
 
-  private static final ThreadLocal<HashData> HASH_DATA = ThreadLocal.withInitial(HashData::new);
+  private static final ThreadLocal<HashData> HASH_DATA = ThreadLocalWithInitial.of(HashData::new);
 
   /**
    * Hash binary encoded data. Consistent with
diff --git a/lang/java/avro/src/main/java/org/apache/avro/message/BinaryMessageDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/message/BinaryMessageDecoder.java
index 61d8ef7..d835bd3 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/message/BinaryMessageDecoder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/message/BinaryMessageDecoder.java
@@ -21,6 +21,7 @@ package org.apache.avro.message;
 import org.apache.avro.Schema;
 import org.apache.avro.SchemaNormalization;
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.internal.ThreadLocalWithInitial;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -47,9 +48,9 @@ import java.util.concurrent.ConcurrentHashMap;
  */
 public class BinaryMessageDecoder<D> extends MessageDecoder.BaseDecoder<D> {
 
-  private static final ThreadLocal<byte[]> HEADER_BUFFER = ThreadLocal.withInitial(() -> new byte[10]);
+  private static final ThreadLocal<byte[]> HEADER_BUFFER = ThreadLocalWithInitial.of(() -> new byte[10]);
 
-  private static final ThreadLocal<ByteBuffer> FP_BUFFER = ThreadLocal.withInitial(() -> {
+  private static final ThreadLocal<ByteBuffer> FP_BUFFER = ThreadLocalWithInitial.of(() -> {
     byte[] header = HEADER_BUFFER.get();
     return ByteBuffer.wrap(header).order(ByteOrder.LITTLE_ENDIAN);
   });
diff --git a/lang/java/avro/src/main/java/org/apache/avro/message/MessageDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/message/MessageDecoder.java
index 8a1e852..007dfed 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/message/MessageDecoder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/message/MessageDecoder.java
@@ -21,6 +21,8 @@ package org.apache.avro.message;
 
 import org.apache.avro.util.ReusableByteArrayInputStream;
 import org.apache.avro.util.ReusableByteBufferInputStream;
+import org.apache.avro.util.internal.ThreadLocalWithInitial;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
@@ -111,11 +113,11 @@ public interface MessageDecoder<D> {
    */
   abstract class BaseDecoder<D> implements MessageDecoder<D> {
 
-    private static final ThreadLocal<ReusableByteArrayInputStream> BYTE_ARRAY_IN = ThreadLocal
-        .withInitial(ReusableByteArrayInputStream::new);
+    private static final ThreadLocal<ReusableByteArrayInputStream> BYTE_ARRAY_IN = ThreadLocalWithInitial
+        .of(ReusableByteArrayInputStream::new);
 
-    private static final ThreadLocal<ReusableByteBufferInputStream> BYTE_BUFFER_IN = ThreadLocal
-        .withInitial(ReusableByteBufferInputStream::new);
+    private static final ThreadLocal<ReusableByteBufferInputStream> BYTE_BUFFER_IN = ThreadLocalWithInitial
+        .of(ReusableByteBufferInputStream::new);
 
     @Override
     public D decode(InputStream stream) throws IOException {
diff --git a/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java
index 093783e..4df0d4c 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java
@@ -24,6 +24,8 @@ import org.apache.avro.generic.GenericData;
 import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.io.DatumWriter;
 import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.util.internal.ThreadLocalWithInitial;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -37,7 +39,7 @@ import java.nio.ByteBuffer;
  */
 public class RawMessageEncoder<D> implements MessageEncoder<D> {
 
-  private static final ThreadLocal<BufferOutputStream> TEMP = ThreadLocal.withInitial(BufferOutputStream::new);
+  private static final ThreadLocal<BufferOutputStream> TEMP = ThreadLocalWithInitial.of(BufferOutputStream::new);
 
   private static final ThreadLocal<BinaryEncoder> ENCODER = new ThreadLocal<>();
 
diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java b/lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java
index fcae137..21c52bc 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java
@@ -30,7 +30,9 @@ import java.util.Map;
  *
  * @param <K> Key of the map-entry
  * @param <V> Value of the map-entry
+ * @deprecated Use org.apache.avro.util.MapEntry
  */
+@Deprecated
 public class MapEntry<K, V> implements Map.Entry<K, V> {
 
   K key;
diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java
index 3dc53be..05d9366 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java
@@ -29,6 +29,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.io.Encoder;
 import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.avro.util.MapEntry;
 
 /**
  * {@link org.apache.avro.io.DatumWriter DatumWriter} for existing classes via
diff --git a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
index 104b12e..5b53939 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
@@ -30,6 +30,7 @@ import org.apache.avro.io.DatumWriter;
 import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.util.ClassUtils;
+import org.apache.avro.util.internal.ClassValueCache;
 
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
@@ -47,6 +48,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.WeakHashMap;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
 
 /** Utilities for generated Java classes and interfaces. */
 public class SpecificData extends GenericData {
@@ -55,36 +57,31 @@ public class SpecificData extends GenericData {
 
   private static final Class<?>[] NO_ARG = new Class[] {};
   private static final Class<?>[] SCHEMA_ARG = new Class[] { Schema.class };
-  private static final ClassValue<Constructor> CTOR_CACHE = new ClassValue<Constructor>() {
-    @Override
-    protected Constructor computeValue(Class<?> c) {
-      boolean useSchema = SchemaConstructable.class.isAssignableFrom(c);
-      try {
-        Constructor meth = c.getDeclaredConstructor(useSchema ? SCHEMA_ARG : NO_ARG);
-        meth.setAccessible(true);
-        return meth;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
+
+  private static final Function<Class<?>, Constructor<?>> CTOR_CACHE = new ClassValueCache<>(c -> {
+    boolean useSchema = SchemaConstructable.class.isAssignableFrom(c);
+    try {
+      Constructor<?> meth = c.getDeclaredConstructor(useSchema ? SCHEMA_ARG : NO_ARG);
+      meth.setAccessible(true);
+      return meth;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
     }
+  });
 
-  };
-  private static final ClassValue<SpecificData> MODEL_CACHE = new ClassValue<SpecificData>() {
-    @Override
-    protected SpecificData computeValue(Class<?> type) {
-      Field specificDataField;
-      try {
-        specificDataField = type.getDeclaredField("MODEL$");
-        specificDataField.setAccessible(true);
-        return (SpecificData) specificDataField.get(null);
-      } catch (NoSuchFieldException e) {
-        // Return default instance
-        return SpecificData.get();
-      } catch (IllegalAccessException e) {
-        throw new AvroRuntimeException("while trying to access field MODEL$ on " + type.getCanonicalName(), e);
-      }
+  private static final Function<Class<?>, SpecificData> MODEL_CACHE = new ClassValueCache<>(c -> {
+    Field specificDataField;
+    try {
+      specificDataField = c.getDeclaredField("MODEL$");
+      specificDataField.setAccessible(true);
+      return (SpecificData) specificDataField.get(null);
+    } catch (NoSuchFieldException e) {
+      // Return default instance
+      return SpecificData.get();
+    } catch (IllegalAccessException e) {
+      throw new AvroRuntimeException("while trying to access field MODEL$ on " + c.getCanonicalName(), e);
     }
-  };
+  });
 
   public static final String CLASS_PROP = "java-class";
   public static final String KEY_CLASS_PROP = "java-key-class";
@@ -183,7 +180,7 @@ public class SpecificData extends GenericData {
    */
   public static <T> SpecificData getForClass(Class<T> c) {
     if (SpecificRecordBase.class.isAssignableFrom(c)) {
-      return MODEL_CACHE.get(c);
+      return MODEL_CACHE.apply(c);
     }
     return SpecificData.get();
   }
@@ -336,12 +333,7 @@ public class SpecificData extends GenericData {
 
   // cache for schemas created from Class objects. Use ClassValue to avoid
   // locking classloaders and is GC and thread safe.
-  private final ClassValue<Schema> schemaClassCache = new ClassValue<Schema>() {
-    @Override
-    protected Schema computeValue(Class<?> type) {
-      return createSchema(type, new HashMap<>());
-    }
-  };
+  private final ClassValueCache<Schema> schemaClassCache = new ClassValueCache<>(c -> createSchema(c, new HashMap<>()));
   // for non-class objects, use a WeakHashMap, but this needs a sync block around
   // it
   private final Map<java.lang.reflect.Type, Schema> schemaTypeCache = Collections.synchronizedMap(new WeakHashMap<>());
@@ -350,7 +342,7 @@ public class SpecificData extends GenericData {
   public Schema getSchema(java.lang.reflect.Type type) {
     try {
       if (type instanceof Class) {
-        return schemaClassCache.get((Class<?>) type);
+        return schemaClassCache.apply((Class<?>) type);
       }
       return schemaTypeCache.computeIfAbsent(type, t -> createSchema(t, new HashMap<>()));
     } catch (Exception e) {
@@ -474,7 +466,7 @@ public class SpecificData extends GenericData {
     boolean useSchema = SchemaConstructable.class.isAssignableFrom(c);
     Object result;
     try {
-      Constructor meth = CTOR_CACHE.get(c);
+      Constructor<?> meth = CTOR_CACHE.apply(c);
       result = meth.newInstance(useSchema ? new Object[] { s } : null);
     } catch (Exception e) {
       throw new RuntimeException(e);
@@ -512,7 +504,7 @@ public class SpecificData extends GenericData {
     }
 
     boolean useSchema = SchemaConstructable.class.isAssignableFrom(c);
-    Constructor meth = (Constructor) CTOR_CACHE.get(c);
+    Constructor<?> meth = CTOR_CACHE.apply(c);
     Object[] params = useSchema ? new Object[] { schema } : (Object[]) null;
 
     return (old, sch) -> {
diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java b/lang/java/avro/src/main/java/org/apache/avro/util/MapEntry.java
similarity index 79%
copy from lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java
copy to lang/java/avro/src/main/java/org/apache/avro/util/MapEntry.java
index fcae137..ec47c45 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/reflect/MapEntry.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/util/MapEntry.java
@@ -15,16 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.avro.reflect;
+package org.apache.avro.util;
 
 import java.util.Map;
 
 /**
- * Class to make Avro immune from the naming variations of key/value fields
- * among several {@link java.util.Map.Entry} implementations. If objects of this
- * class are used instead of the regular ones obtained by
- * {@link Map#entrySet()}, then we need not worry about the actual field-names
- * or any changes to them in the future.<BR>
+ * An implementation of {@link Map.Entry} with well-defined member names.
+ * <p>
+ * Using this class helps make Avro immune from the naming variations of
+ * key/value fields among several {@link Map.Entry} implementations. If objects
+ * of this class are used instead of the regular ones obtained by
+ * {@link Map#entrySet()}, then we need not worry about the actual field names
+ * or any changes to them in the future.
+ * <p>
  * Example: {@code ConcurrentHashMap.MapEntry} does not name the fields as key/
  * value in Java 1.8 while it used to do so in Java 1.7
  *
diff --git a/lang/java/avro/src/main/java/org/apache/avro/util/internal/ClassValueCache.java b/lang/java/avro/src/main/java/org/apache/avro/util/internal/ClassValueCache.java
new file mode 100644
index 0000000..25cc3fb
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/util/internal/ClassValueCache.java
@@ -0,0 +1,47 @@
+/*
+ * 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
+ *
+ *     https://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.avro.util.internal;
+
+import java.util.function.Function;
+
+/**
+ * Wraps a {@link ClassValue} cache so it can be overridden in an android
+ * environment, where it isn't available.
+ *
+ * @param <R> Return type of the ClassValue
+ */
+public class ClassValueCache<R> implements Function<Class<?>, R> {
+
+  private final Function<Class<?>, R> ifAbsent;
+
+  private final ClassValue<R> cache = new ClassValue<R>() {
+    @Override
+    protected R computeValue(Class<?> c) {
+      return ifAbsent.apply(c);
+    }
+  };
+
+  public ClassValueCache(Function<Class<?>, R> ifAbsent) {
+    this.ifAbsent = ifAbsent;
+  }
+
+  @Override
+  public R apply(Class<?> c) {
+    return cache.get(c);
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/util/internal/ThreadLocalWithInitial.java b/lang/java/avro/src/main/java/org/apache/avro/util/internal/ThreadLocalWithInitial.java
new file mode 100644
index 0000000..a49267b
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/util/internal/ThreadLocalWithInitial.java
@@ -0,0 +1,32 @@
+/*
+ * 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
+ *
+ *     https://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.avro.util.internal;
+
+import java.util.function.Supplier;
+
+/**
+ * Wraps a {@link ThreadLocal#withInitial(Supplier)} so it can be overridden in
+ * an android environment, where this method is not available until API 26.
+ */
+public class ThreadLocalWithInitial {
+
+  /** Delegate a ThreadLocal instance with the supplier. */
+  public static <T> ThreadLocal<T> of(Supplier<? extends T> supplier) {
+    return ThreadLocal.withInitial(supplier);
+  }
+}
diff --git a/lang/java/avro/src/test/java/org/apache/avro/util/internal/TestClassValueCache.java b/lang/java/avro/src/test/java/org/apache/avro/util/internal/TestClassValueCache.java
new file mode 100644
index 0000000..d2c9217
--- /dev/null
+++ b/lang/java/avro/src/test/java/org/apache/avro/util/internal/TestClassValueCache.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://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.avro.util.internal;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.*;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public class TestClassValueCache {
+
+  @Test
+  public void testBasic() {
+    ClassValueCache<String> cache = new ClassValueCache<>(Class::toString);
+
+    String fromCache = cache.apply(String.class);
+    assertThat(fromCache, is("class java.lang.String"));
+
+    assertThat(cache.apply(String.class), sameInstance(fromCache));
+  }
+
+}
diff --git a/lang/java/pom.xml b/lang/java/pom.xml
index 7e3796d..f96ce29 100644
--- a/lang/java/pom.xml
+++ b/lang/java/pom.xml
@@ -68,6 +68,7 @@
   </properties>
 
   <modules>
+    <module>android</module>
     <module>avro</module>
     <module>compiler</module>
     <module>maven-plugin</module>