You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/01/23 01:30:23 UTC

[05/50] [abbrv] calcite git commit: [CALCITE-915] Tests should unset ThreadLocal values on exit

[CALCITE-915] Tests should unset ThreadLocal values on exit


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

Branch: refs/remotes/julianhyde/master
Commit: c104c75bdf6d663070cf16b5ab0f94f8c269c9b6
Parents: 7837e64
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Oct 8 16:23:20 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/prepare/Prepare.java     |  15 +-
 .../org/apache/calcite/util/TryThreadLocal.java |  71 ++++++++
 .../java/org/apache/calcite/test/JdbcTest.java  | 180 +++++++++----------
 .../calcite/test/MaterializationTest.java       |  21 +--
 .../java/org/apache/calcite/util/UtilTest.java  |  37 ++++
 5 files changed, 212 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/Prepare.java b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
index afb722f..3530e93 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -52,6 +52,7 @@ import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.Programs;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.trace.CalciteTimingTracer;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -83,12 +84,14 @@ public abstract class Prepare {
   protected RelDataType parameterRowType;
 
   // temporary. for testing.
-  public static final ThreadLocal<Boolean> THREAD_TRIM =
-      new ThreadLocal<Boolean>() {
-        @Override protected Boolean initialValue() {
-          return false;
-        }
-      };
+  public static final TryThreadLocal<Boolean> THREAD_TRIM =
+      TryThreadLocal.of(false);
+
+  /** Temporary, while CALCITE-816 is under development.
+   *
+   * @see org.apache.calcite.util.Util#deprecated(Object, boolean) */
+  public static final TryThreadLocal<Boolean> THREAD_EXPAND =
+      TryThreadLocal.of(false);
 
   public Prepare(CalcitePrepare.Context context, CatalogReader catalogReader,
       Convention resultConvention) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java b/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java
new file mode 100644
index 0000000..b278174
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/TryThreadLocal.java
@@ -0,0 +1,71 @@
+/*
+ * 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.calcite.util;
+
+/**
+ * Thread-local variable that returns a handle that can be closed.
+ *
+ * @param <T> Value type
+ */
+public class TryThreadLocal<T> extends ThreadLocal<T> {
+  private final T initialValue;
+
+  /** Creates a TryThreadLocal.
+   *
+   * @param initialValue Initial value
+   */
+  public static <T> TryThreadLocal<T> of(T initialValue) {
+    return new TryThreadLocal<>(initialValue);
+  }
+
+  private TryThreadLocal(T initialValue) {
+    this.initialValue = initialValue;
+  }
+
+  // It is important that this method is final.
+  // This ensures that the sub-class does not choose a different initial
+  // value. Then the close logic can detect whether the previous value was
+  // equal to the initial value.
+  @Override protected final T initialValue() {
+    return initialValue;
+  }
+
+  /** Assigns the value as {@code value} for the current thread.
+   * Returns a {@link Memo} which, when closed, will assign the value
+   * back to the previous value. */
+  public Memo push(T value) {
+    final T previous = get();
+    set(value);
+    return new Memo() {
+      public void close() {
+        if (previous == initialValue) {
+          remove();
+        } else {
+          set(previous);
+        }
+      }
+    };
+  }
+
+  /** Remembers to set the value back. */
+  public interface Memo extends AutoCloseable {
+    /** Sets the value back; never throws. */
+    @Override void close();
+  }
+}
+
+// End TryThreadLocal.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index b64cba4..243f903 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -91,6 +91,7 @@ import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Smalls;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -247,8 +248,8 @@ public class JdbcTest {
   @Test public void testModelWithModifiableView() throws Exception {
     final List<Employee> employees = new ArrayList<>();
     employees.add(new Employee(135, 10, "Simon", 56.7f, null));
-    try {
-      EmpDeptTableFactory.THREAD_COLLECTION.set(employees);
+    try (final TryThreadLocal.Memo ignore =
+             EmpDeptTableFactory.THREAD_COLLECTION.push(employees)) {
       final CalciteAssert.AssertThat with = modelWithView(
           "select \"name\", \"empid\" as e, \"salary\" "
               + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" = 10",
@@ -316,8 +317,6 @@ public class JdbcTest {
               }
             }
           });
-    } finally {
-      EmpDeptTableFactory.THREAD_COLLECTION.remove();
     }
   }
 
@@ -325,9 +324,8 @@ public class JdbcTest {
   @Test public void testModelWithInvalidModifiableView() throws Exception {
     final List<Employee> employees = new ArrayList<>();
     employees.add(new Employee(135, 10, "Simon", 56.7f, null));
-    try {
-      EmpDeptTableFactory.THREAD_COLLECTION.set(employees);
-
+    try (final TryThreadLocal.Memo ignore =
+             EmpDeptTableFactory.THREAD_COLLECTION.push(employees)) {
       Util.discard(RESOURCE.noValueSuppliedForViewColumn(null, null));
       modelWithView("select \"name\", \"empid\" as e, \"salary\" "
               + "from \"MUTABLE_EMPLOYEES\" where \"commission\" = 10",
@@ -397,8 +395,6 @@ public class JdbcTest {
           null)
           .query("select \"name\" from \"adhoc\".V order by \"name\"")
           .runs();
-    } finally {
-      EmpDeptTableFactory.THREAD_COLLECTION.remove();
     }
   }
 
@@ -871,67 +867,67 @@ public class JdbcTest {
   @Test public void testOnConnectionClose() throws Exception {
     final int[] closeCount = {0};
     final int[] statementCloseCount = {0};
-    HandlerDriver.HANDLERS.set(
-        new HandlerImpl() {
-          @Override public void
-          onConnectionClose(AvaticaConnection connection) {
-            ++closeCount[0];
-            throw new RuntimeException();
-          }
-          @Override public void onStatementClose(AvaticaStatement statement) {
-            ++statementCloseCount[0];
-            throw new RuntimeException();
-          }
-        });
-    final HandlerDriver driver =
-        new HandlerDriver();
-    CalciteConnection connection = (CalciteConnection)
-        driver.connect("jdbc:calcite:", new Properties());
-    SchemaPlus rootSchema = connection.getRootSchema();
-    rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
-    connection.setSchema("hr");
-    final Statement statement = connection.createStatement();
-    final ResultSet resultSet =
-        statement.executeQuery("select * from \"emps\"");
-    assertEquals(0, closeCount[0]);
-    assertEquals(0, statementCloseCount[0]);
-    resultSet.close();
-    try {
-      resultSet.next();
-      fail("resultSet.next() should throw SQLException when closed");
-    } catch (SQLException e) {
-      assertThat(e.getMessage(),
-          containsString("next() called on closed cursor"));
-    }
-    assertEquals(0, closeCount[0]);
-    assertEquals(0, statementCloseCount[0]);
+    final HandlerImpl h = new HandlerImpl() {
+      @Override public void onConnectionClose(AvaticaConnection connection) {
+        ++closeCount[0];
+        throw new RuntimeException();
+      }
 
-    // Close statement. It throws SQLException, but statement is still closed.
-    try {
-      statement.close();
-      fail("expecting error");
-    } catch (SQLException e) {
-      // ok
-    }
-    assertEquals(0, closeCount[0]);
-    assertEquals(1, statementCloseCount[0]);
+      @Override public void onStatementClose(AvaticaStatement statement) {
+        ++statementCloseCount[0];
+        throw new RuntimeException();
+      }
+    };
+    try (final TryThreadLocal.Memo ignore =
+             HandlerDriver.HANDLERS.push(h)) {
+      final HandlerDriver driver = new HandlerDriver();
+      CalciteConnection connection = (CalciteConnection)
+          driver.connect("jdbc:calcite:", new Properties());
+      SchemaPlus rootSchema = connection.getRootSchema();
+      rootSchema.add("hr", new ReflectiveSchema(new HrSchema()));
+      connection.setSchema("hr");
+      final Statement statement = connection.createStatement();
+      final ResultSet resultSet =
+          statement.executeQuery("select * from \"emps\"");
+      assertEquals(0, closeCount[0]);
+      assertEquals(0, statementCloseCount[0]);
+      resultSet.close();
+      try {
+        resultSet.next();
+        fail("resultSet.next() should throw SQLException when closed");
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            containsString("next() called on closed cursor"));
+      }
+      assertEquals(0, closeCount[0]);
+      assertEquals(0, statementCloseCount[0]);
 
-    // Close connection. It throws SQLException, but connection is still closed.
-    try {
-      connection.close();
-      fail("expecting error");
-    } catch (SQLException e) {
-      // ok
-    }
-    assertEquals(1, closeCount[0]);
-    assertEquals(1, statementCloseCount[0]);
+      // Close statement. It throws SQLException, but statement is still closed.
+      try {
+        statement.close();
+        fail("expecting error");
+      } catch (SQLException e) {
+        // ok
+      }
+      assertEquals(0, closeCount[0]);
+      assertEquals(1, statementCloseCount[0]);
 
-    // Close a closed connection. Handler is not called again.
-    connection.close();
-    assertEquals(1, closeCount[0]);
-    assertEquals(1, statementCloseCount[0]);
+      // Close connection. It throws SQLException, but connection is still closed.
+      try {
+        connection.close();
+        fail("expecting error");
+      } catch (SQLException e) {
+        // ok
+      }
+      assertEquals(1, closeCount[0]);
+      assertEquals(1, statementCloseCount[0]);
+
+      // Close a closed connection. Handler is not called again.
+      connection.close();
+      assertEquals(1, closeCount[0]);
+      assertEquals(1, statementCloseCount[0]);
 
-    HandlerDriver.HANDLERS.remove();
+    }
   }
 
   /** Tests {@link java.sql.Statement}.{@code closeOnCompletion()}. */
@@ -3335,15 +3331,18 @@ public class JdbcTest {
 
   /** Query that reads no columns from either underlying table. */
   @Test public void testCountStar() {
-    CalciteAssert.hr()
-        .query("select count(*) c from \"hr\".\"emps\", \"hr\".\"depts\"")
-        .convertContains("LogicalAggregate(group=[{}], C=[COUNT()])\n"
-            + "  LogicalProject(DUMMY=[0])\n"
-            + "    LogicalJoin(condition=[true], joinType=[inner])\n"
-            + "      LogicalProject(DUMMY=[0])\n"
-            + "        EnumerableTableScan(table=[[hr, emps]])\n"
-            + "      LogicalProject(DUMMY=[0])\n"
-            + "        EnumerableTableScan(table=[[hr, depts]])");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true);
+         final TryThreadLocal.Memo memo = Prepare.THREAD_EXPAND.push(true)) {
+      CalciteAssert.hr()
+          .query("select count(*) c from \"hr\".\"emps\", \"hr\".\"depts\"")
+          .convertContains("LogicalAggregate(group=[{}], C=[COUNT()])\n"
+              + "  LogicalProject(DUMMY=[0])\n"
+              + "    LogicalJoin(condition=[true], joinType=[inner])\n"
+              + "      LogicalProject(DUMMY=[0])\n"
+              + "        EnumerableTableScan(table=[[hr, emps]])\n"
+              + "      LogicalProject(DUMMY=[0])\n"
+              + "        EnumerableTableScan(table=[[hr, depts]])");
+    }
   }
 
   /** Same result (and plan) as {@link #testSelectDistinct}. */
@@ -4173,26 +4172,25 @@ public class JdbcTest {
 
   /** Tests that field-trimming creates a project near the table scan. */
   @Test public void testTrimFields() throws Exception {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       CalciteAssert.hr()
           .query("select \"name\", count(\"commission\") + 1\n"
-            + "from \"hr\".\"emps\"\n"
-            + "group by \"deptno\", \"name\"")
+              + "from \"hr\".\"emps\"\n"
+              + "group by \"deptno\", \"name\"")
           .convertContains("LogicalProject(name=[$1], EXPR$1=[+($2, 1)])\n"
               + "  LogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)])\n"
               + "    LogicalProject(deptno=[$1], name=[$2], commission=[$4])\n"
               + "      EnumerableTableScan(table=[[hr, emps]])\n");
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
   /** Tests that field-trimming creates a project near the table scan, in a
    * query with windowed-aggregation. */
   @Test public void testTrimFieldsOver() throws Exception {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true);
+         final TryThreadLocal.Memo memo = Prepare.THREAD_EXPAND.push(true)) {
+      Util.discard(memo);
+      // The correct plan has a project on a filter on a project on a scan.
       CalciteAssert.hr()
           .query("select \"name\",\n"
               + "  count(\"commission\") over (partition by \"deptno\") + 1\n"
@@ -4203,8 +4201,6 @@ public class JdbcTest {
               + "  LogicalFilter(condition=[>($0, 10)])\n"
               + "    LogicalProject(empid=[$0], deptno=[$1], name=[$2], commission=[$4])\n"
               + "      EnumerableTableScan(table=[[hr, emps]])\n");
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -4220,9 +4216,10 @@ public class JdbcTest {
             "M=1",
             "M=1");
   }
+
   /** Tests multiple window aggregates over constants.
    * This tests that EnumerableWindowRel is able to reference the right slot
-   * when accessing constant for aggregation argument.*/
+   * when accessing constant for aggregation argument. */
   @Test public void testWinAggConstantMultipleConstants() {
     CalciteAssert.that()
         .with(CalciteAssert.Config.REGULAR)
@@ -4644,7 +4641,9 @@ public class JdbcTest {
    * use as scratch space during development. */
   // Do not add '@Ignore'; just remember not to commit changes to dummy.iq
   @Test public void testRunDummy() throws Exception {
-    checkRun("sql/dummy.iq");
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(false)) {
+      checkRun("sql/dummy.iq");
+    }
   }
 
   @Test public void testRunAgg() throws Exception {
@@ -6478,8 +6477,8 @@ public class JdbcTest {
 
   /** Factory for EMP and DEPT tables. */
   public static class EmpDeptTableFactory implements TableFactory<Table> {
-    public static final ThreadLocal<List<Employee>> THREAD_COLLECTION =
-        new ThreadLocal<>();
+    public static final TryThreadLocal<List<Employee>> THREAD_COLLECTION =
+        TryThreadLocal.of(null);
 
     public Table create(
         SchemaPlus schema,
@@ -6578,7 +6577,8 @@ public class JdbcTest {
 
   /** Mock driver that a given {@link Handler}. */
   public static class HandlerDriver extends org.apache.calcite.jdbc.Driver {
-    private static final ThreadLocal<Handler> HANDLERS = new ThreadLocal<>();
+    private static final TryThreadLocal<Handler> HANDLERS =
+        TryThreadLocal.of(null);
 
     public HandlerDriver() {
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
index 65aabd7..05726b3 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.JsonBuilder;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.Util;
 
 import org.apache.commons.lang3.StringUtils;
@@ -127,8 +128,7 @@ public class MaterializationTest {
   }
 
   @Test public void testFilterQueryOnProjectView() {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(
@@ -140,8 +140,6 @@ public class MaterializationTest {
           .enableMaterializations(true)
           .explainContains("EnumerableTableScan(table=[[hr, m0]])")
           .sameResultWithMaterializationsDisabled();
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -155,8 +153,7 @@ public class MaterializationTest {
    * definition. */
   private void checkMaterialize(String materialize, String query, String model,
       Function<ResultSet, Void> explainChecker) {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(model, "m0", materialize)
@@ -164,8 +161,6 @@ public class MaterializationTest {
           .enableMaterializations(true)
           .explainMatches("", explainChecker)
           .sameResultWithMaterializationsDisabled();
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -173,16 +168,13 @@ public class MaterializationTest {
    * definition. */
   private void checkNoMaterialize(String materialize, String query,
       String model) {
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(model, "m0", materialize)
           .query(query)
           .enableMaterializations(true)
           .explainContains("EnumerableTableScan(table=[[hr, emps]])");
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 
@@ -876,8 +868,7 @@ public class MaterializationTest {
    * Pre-populated materializations</a>. */
   @Test public void testPrePopulated() {
     String q = "select \"deptno\" from \"emps\"";
-    try {
-      Prepare.THREAD_TRIM.set(true);
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
       CalciteAssert.that()
           .withMaterializations(
@@ -897,8 +888,6 @@ public class MaterializationTest {
           .enableMaterializations(true)
           .explainMatches("", CONTAINS_LOCATIONS)
           .sameResultWithMaterializationsDisabled();
-    } finally {
-      Prepare.THREAD_TRIM.set(false);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c104c75b/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 78202e2..60122fb 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -73,6 +73,7 @@ import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.isA;
 import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.nullValue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
@@ -1472,6 +1473,42 @@ public class UtilTest {
     final String asString = Util.listToString(list);
     assertThat(Util.stringToList(asString), is(list));
   }
+
+  /** Tests {@link org.apache.calcite.util.TryThreadLocal}.
+   *
+   * <p>TryThreadLocal was introduced to fix
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-915">[CALCITE-915]
+   * Tests do not unset ThreadLocal values on exit</a>. */
+  @Test public void testTryThreadLocal() {
+    final TryThreadLocal<String> local1 = TryThreadLocal.of("foo");
+    assertThat(local1.get(), is("foo"));
+    TryThreadLocal.Memo memo1 = local1.push("bar");
+    assertThat(local1.get(), is("bar"));
+    local1.set("baz");
+    assertThat(local1.get(), is("baz"));
+    memo1.close();
+    assertThat(local1.get(), is("foo"));
+
+    final TryThreadLocal<String> local2 = TryThreadLocal.of(null);
+    assertThat(local2.get(), nullValue());
+    TryThreadLocal.Memo memo2 = local2.push("a");
+    assertThat(local2.get(), is("a"));
+    local2.set("b");
+    assertThat(local2.get(), is("b"));
+    TryThreadLocal.Memo memo2B = local2.push(null);
+    assertThat(local2.get(), nullValue());
+    memo2B.close();
+    assertThat(local2.get(), is("b"));
+    memo2.close();
+    assertThat(local2.get(), nullValue());
+
+    local2.set("x");
+    try (TryThreadLocal.Memo ignore = local2.push("y")) {
+      assertThat(local2.get(), is("y"));
+      local2.set("z");
+    }
+    assertThat(local2.get(), is("x"));
+  }
 }
 
 // End UtilTest.java