You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by vl...@apache.org on 2018/08/29 11:15:26 UTC

calcite git commit: [CALCITE-2495] Support encoded URLs in calcite.util.Source, and use it for URL->File conversion in tests

Repository: calcite
Updated Branches:
  refs/heads/master 02794e2b9 -> 2f33a0c57


[CALCITE-2495] Support encoded URLs in calcite.util.Source, and use it for URL->File conversion in tests

For the record:
1) URL.getPath() produces %20, so it is added to forbidden signatures
2) Paths.get(url.toURI()).toFile() almost works, however it fails with URL is not hierarchical for new URL("file:test.java")
3) new File(URL.toURI() is worse than #2
4) URLDecoder must not be used to decode %20, since it will convert + to spaces as well, thus it will corrupt test.c++ filenames
5) It looks like url.toURI().getSchemeSpecificPart()) properly handles "opaque" URIs (which are relative file:test.java kind of URLs)
6) file:/test is relative for Windows, and absolute would be like file:c:/test


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

Branch: refs/heads/master
Commit: 2f33a0c57b7b7e77b8193d0fff1e3531119aee0a
Parents: 02794e2
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Tue Aug 28 14:45:51 2018 +0300
Committer: Vladimir Sitnikov <si...@gmail.com>
Committed: Wed Aug 29 13:21:24 2018 +0300

----------------------------------------------------------------------
 .../calcite/test/CassandraAdapterTest.java      |  6 +-
 .../java/org/apache/calcite/util/Sources.java   | 65 ++++++++++---
 .../calcite/sql/parser/SqlParserTest.java       | 21 +++--
 .../org/apache/calcite/test/CalciteAssert.java  |  7 ++
 .../org/apache/calcite/test/DiffRepository.java |  5 +-
 .../org/apache/calcite/test/QuidemTest.java     | 29 ++----
 .../org/apache/calcite/util/SourceTest.java     | 97 ++++++++++++++++++--
 .../org/apache/calcite/test/DruidAdapterIT.java | 33 ++++---
 .../apache/calcite/test/DruidAdapterIT2.java    | 15 ++-
 .../java/org/apache/calcite/test/CsvTest.java   | 16 +---
 .../apache/calcite/adapter/file/FileReader.java |  3 +-
 .../calcite/adapter/file/FileReaderTest.java    | 13 +--
 .../adapter/geode/rel/BaseGeodeAdapterIT.java   |  9 +-
 .../geode/rel/GeodeAdapterBookshopIT.java       |  5 +-
 .../calcite/adapter/geode/rel/GeodeZipsIT.java  |  5 +-
 .../apache/calcite/test/AbstractPigTest.java    |  9 +-
 .../org/apache/calcite/test/PigAdapterTest.java | 16 +---
 .../calcite/adapter/os/OsAdapterTest.java       |  4 +-
 src/main/config/forbidden-apis/signatures.txt   |  3 +
 19 files changed, 225 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
----------------------------------------------------------------------
diff --git a/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java b/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
index 72caa80..ec67527 100644
--- a/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
+++ b/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
@@ -60,8 +61,9 @@ public class CassandraAdapterTest {
   /** Connection factory based on the "mongo-zips" model. */
   private static final ImmutableMap<String, String> TWISSANDRA =
       ImmutableMap.of("model",
-          CassandraAdapterTest.class.getResource("/model.json")
-              .getPath());
+          Sources.of(
+              CassandraAdapterTest.class.getResource("/model.json"))
+              .file().getAbsolutePath());
 
   /**
    * Whether to run this test.

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/core/src/main/java/org/apache/calcite/util/Sources.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Sources.java b/core/src/main/java/org/apache/calcite/util/Sources.java
index ae6e9da..c50cd9c 100644
--- a/core/src/main/java/org/apache/calcite/util/Sources.java
+++ b/core/src/main/java/org/apache/calcite/util/Sources.java
@@ -23,8 +23,11 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.Reader;
 import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
 import java.util.Objects;
 import java.util.zip.GZIPInputStream;
 
@@ -38,6 +41,10 @@ public abstract class Sources {
     return new FileSource(file);
   }
 
+  public static Source of(URL url) {
+    return new FileSource(url);
+  }
+
   public static Source file(File baseDirectory, String fileName) {
     final File file = new File(fileName);
     if (baseDirectory != null && !file.isAbsolute()) {
@@ -49,9 +56,8 @@ public abstract class Sources {
 
   public static Source url(String url) {
     try {
-      URL url_ = new URL(url);
-      return new FileSource(url_);
-    } catch (MalformedURLException e) {
+      return of(new URL(url));
+    } catch (MalformedURLException | IllegalArgumentException e) {
       throw new RuntimeException("Malformed URL: '" + url + "'", e);
     }
   }
@@ -76,11 +82,7 @@ public abstract class Sources {
 
     private FileSource(URL url) {
       this.url = Objects.requireNonNull(url);
-      if (url.getProtocol().equals("file")) {
-        this.file = new File(url.getFile());
-      } else {
-        this.file = null;
-      }
+      this.file = urlToFile(url);
     }
 
     private FileSource(File file) {
@@ -88,6 +90,25 @@ public abstract class Sources {
       this.url = null;
     }
 
+    private File urlToFile(URL url) {
+      if (!"file".equals(url.getProtocol())) {
+        return null;
+      }
+      URI uri;
+      try {
+        uri = url.toURI();
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException("Unable to convert URL " + url + " to URI", e);
+      }
+      if (uri.isOpaque()) {
+        // It is like file:test%20file.c++
+        // getSchemeSpecificPart would return "test file.c++"
+        return new File(uri.getSchemeSpecificPart());
+      }
+      // See https://stackoverflow.com/a/17870390/1261287
+      return Paths.get(uri).toFile();
+    }
+
     @Override public String toString() {
       return (url != null ? url : file).toString();
     }
@@ -111,7 +132,15 @@ public abstract class Sources {
     }
 
     public String path() {
-      return file != null ? file.getPath() : url.toExternalForm();
+      if (file != null) {
+        return file.getPath();
+      }
+      try {
+        // Decode %20 and friends
+        return url.toURI().getSchemeSpecificPart();
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException("Unable to convert URL " + url + " to URI", e);
+      }
     }
 
     public Reader reader() throws IOException {
@@ -149,20 +178,26 @@ public abstract class Sources {
     }
 
     public Source append(Source child) {
-      String path;
       if (isFile(child)) {
-        path = child.file().getPath();
         if (child.file().isAbsolute()) {
           return child;
         }
       } else {
-        path = child.url().getPath();
-        if (path.startsWith("/")) {
-          return child;
+        try {
+          URI uri = child.url().toURI();
+          if (!uri.isOpaque()) {
+            // The URL is "absolute" (it starts with a slash)
+            return child;
+          }
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException("Unable to convert URL " + child.url() + " to URI", e);
         }
       }
+      String path = child.path();
       if (url != null) {
-        return Sources.url(url + "/" + path);
+        String encodedPath = new File(".").toURI().relativize(new File(path).toURI())
+            .getRawSchemeSpecificPart();
+        return Sources.url(url + "/" + encodedPath);
       } else {
         return Sources.file(file, path);
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index c3459f0..4db60f1 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.test.DiffTestCase;
 import org.apache.calcite.test.SqlValidatorTestCase;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ConversionUtil;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
@@ -48,8 +49,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.URL;
-import java.net.URLDecoder;
+import java.nio.file.Paths;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
@@ -7152,14 +7152,15 @@ public class SqlParserTest {
     assumeTrue("don't run this test for sub-classes", isNotSubclass());
     // inUrl = "file:/home/x/calcite/core/target/test-classes/hsqldb-model.json"
     String path = "hsqldb-model.json";
-    final URL inUrl = SqlParserTest.class.getResource("/" + path);
-    // URL will convert spaces to %20, undo that
-    String x = URLDecoder.decode(inUrl.getFile(), "UTF-8");
-    assert x.endsWith(path);
-    x = x.substring(0, x.length() - path.length());
-    assert x.endsWith("core/target/test-classes/");
-    x = x.substring(0, x.length() - "core/target/test-classes/".length());
-    final File base = new File(x);
+    File hsqlDbModel = Sources.of(SqlParserTest.class.getResource("/" + path)).file();
+    assert hsqlDbModel.getAbsolutePath().endsWith(
+        Paths.get("core", "target", "test-classes", "hsqldb-model.json").toString())
+        : hsqlDbModel.getAbsolutePath()
+        + " should end with core/target/test-classes/hsqldb-model.json";
+    // skip hsqldb-model.json, test-classes, target, core
+    // The assertion above protects us from walking over unrelated paths
+    final File base = hsqlDbModel.getAbsoluteFile()
+        .getParentFile().getParentFile().getParentFile().getParentFile();
     final File inFile = new File(base, "site/_docs/reference.md");
     final File outFile = new File(base, "core/target/surefire/reference.md");
     outFile.getParentFile().mkdirs();

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 3d7ce4e..f561124 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -52,6 +52,7 @@ import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Smalls;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 
 import org.apache.commons.dbcp2.PoolableConnectionFactory;
@@ -75,6 +76,7 @@ import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.net.URL;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -950,6 +952,11 @@ public class CalciteAssert {
       return with(CalciteConnectionProperty.MODEL, "inline:" + model);
     }
 
+    public final AssertThat withModel(URL model) {
+      return with(CalciteConnectionProperty.MODEL,
+          Sources.of(model).file().getAbsolutePath());
+    }
+
     public final AssertThat withMaterializations(String model,
          final String... materializations) {
       return withMaterializations(model, false, materializations);

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/core/src/test/java/org/apache/calcite/test/DiffRepository.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepository.java b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
index ffd3fa6..075a91e 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffRepository.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
@@ -18,6 +18,7 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.XmlOutput;
 
@@ -746,7 +747,9 @@ public class DiffRepository {
     if (diffRepository == null) {
       final URL refFile = findFile(clazz, ".xml");
       final File logFile =
-          new File(refFile.getFile().replace("test-classes", "surefire"));
+          new File(
+              Sources.of(refFile).file().getAbsolutePath()
+                  .replace("test-classes", "surefire"));
       diffRepository =
           new DiffRepository(refFile, logFile, baseRepository, filter);
       MAP_CLASS_TO_REPOSITORY.put(clazz, diffRepository);

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
index 6bf727f..9f089ce 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Closer;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.Lists;
@@ -53,7 +54,6 @@ import java.sql.DriverManager;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.function.Function;
 
 import static org.junit.Assert.fail;
 
@@ -107,22 +107,14 @@ public abstract class QuidemTest {
 
   protected static Collection<Object[]> data(String first) {
     // inUrl = "file:/home/fred/calcite/core/target/test-classes/sql/agg.iq"
-    final URL inUrl = JdbcTest.class.getResource("/" + first);
-    String x = inUrl.getFile();
-    assert x.endsWith(first);
-    final String base =
-        File.separatorChar == '\\'
-            ? x.substring(1, x.length() - first.length())
-                .replace('/', File.separatorChar)
-            : x.substring(0, x.length() - first.length());
-    final File firstFile = new File(x);
+    final URL inUrl = JdbcTest.class.getResource("/" + n2u(first));
+    final File firstFile = Sources.of(inUrl).file();
+    final int commonPrefixLength = firstFile.getAbsolutePath().length() - first.length();
     final File dir = firstFile.getParentFile();
     final List<String> paths = new ArrayList<>();
     final FilenameFilter filter = new PatternFilenameFilter(".*\\.iq$");
     for (File f : Util.first(dir.listFiles(filter), new File[0])) {
-      assert f.getAbsolutePath().startsWith(base)
-          : "f: " + f.getAbsolutePath() + "; base: " + base;
-      paths.add(f.getAbsolutePath().substring(base.length()));
+      paths.add(f.getAbsolutePath().substring(commonPrefixLength));
     }
     return Lists.transform(paths, path -> new Object[] {path});
   }
@@ -139,15 +131,8 @@ public abstract class QuidemTest {
       // e.g. path = "sql/outer.iq"
       // inUrl = "file:/home/fred/calcite/core/target/test-classes/sql/outer.iq"
       final URL inUrl = JdbcTest.class.getResource("/" + n2u(path));
-      String x = u2n(inUrl.getFile());
-      assert x.endsWith(path)
-          : "x: " + x + "; path: " + path;
-      x = x.substring(0, x.length() - path.length());
-      assert x.endsWith(u2n("/test-classes/"));
-      x = x.substring(0, x.length() - u2n("/test-classes/").length());
-      final File base = new File(x);
-      inFile = new File(base, u2n("/test-classes/") + path);
-      outFile = new File(base, u2n("/surefire/") + path);
+      inFile = Sources.of(inUrl).file();
+      outFile = new File(inFile.getAbsoluteFile().getParent(), u2n("surefire/") + path);
     }
     Util.discard(outFile.getParentFile().mkdirs());
     try (final Reader reader = Util.reader(inFile);

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/core/src/test/java/org/apache/calcite/util/SourceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/SourceTest.java b/core/src/test/java/org/apache/calcite/util/SourceTest.java
index 8473c36..6789877 100644
--- a/core/src/test/java/org/apache/calcite/util/SourceTest.java
+++ b/core/src/test/java/org/apache/calcite/util/SourceTest.java
@@ -20,25 +20,104 @@ import org.junit.Test;
 
 import java.io.File;
 
+import static org.apache.calcite.util.Sources.file;
+import static org.apache.calcite.util.Sources.url;
+
 import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
 /**
  * Tests for {@link Source}.
  */
 public class SourceTest {
-  @Test public void testAppend() {
-    final Source foo = Sources.file(null, "/foo");
-    final Source bar = Sources.file(null, "bar");
-    final Source fooBar = foo.append(bar);
-    assertThat(fooBar.file().toString(),
-        is("/foo/bar".replace('/', File.separatorChar)));
+  private static final String ROOT_PREFIX = getRootPrefix();
+
+  private static String getRootPrefix() {
+    for (String s : new String[]{"/", "c:/"}) {
+      if (new File(s).isAbsolute()) {
+        return s;
+      }
+    }
+    throw new IllegalStateException(
+        "Unsupported operation system detected. Both / and c:/ produce relative paths");
+  }
+
+  @Test public void testAppendWithSpaces() {
+    String fooRelative = "fo o+";
+    String fooAbsolute = ROOT_PREFIX + "fo o+";
+    String barRelative = "b ar+";
+    String barAbsolute = ROOT_PREFIX + "b ar+";
+    assertAppend(file(null, fooRelative), file(null, barRelative), "fo o+/b ar+");
+    assertAppend(file(null, fooRelative), file(null, barAbsolute), barAbsolute);
+    assertAppend(file(null, fooAbsolute), file(null, barRelative), ROOT_PREFIX + "fo o+/b ar+");
+    assertAppend(file(null, fooAbsolute), file(null, barAbsolute), barAbsolute);
+
+    String urlFooRelative = "file:fo%20o+";
+    String urlFooAbsolute = "file:" + ROOT_PREFIX + "fo%20o+";
+    String urlBarRelative = "file:b%20ar+";
+    String urlBarAbsolute = "file:" + ROOT_PREFIX + "b%20ar+";
+    assertAppend(url(urlFooRelative), url(urlBarRelative), "fo o+/b ar+");
+    assertAppend(url(urlFooRelative), url(urlBarAbsolute), barAbsolute);
+    assertAppend(url(urlFooAbsolute), url(urlBarRelative), ROOT_PREFIX + "fo o+/b ar+");
+    assertAppend(url(urlFooAbsolute), url(urlBarAbsolute), barAbsolute);
+
+    assertAppend(file(null, fooRelative), url(urlBarRelative), "fo o+/b ar+");
+    assertAppend(file(null, fooRelative), url(urlBarAbsolute), barAbsolute);
+    assertAppend(file(null, fooAbsolute), url(urlBarRelative), ROOT_PREFIX + "fo o+/b ar+");
+    assertAppend(file(null, fooAbsolute), url(urlBarAbsolute), barAbsolute);
+
+    assertAppend(url(urlFooRelative), file(null, barRelative), "fo o+/b ar+");
+    assertAppend(url(urlFooRelative), file(null, barAbsolute), barAbsolute);
+    assertAppend(url(urlFooAbsolute), file(null, barRelative), ROOT_PREFIX + "fo o+/b ar+");
+    assertAppend(url(urlFooAbsolute), file(null, barAbsolute), barAbsolute);
+  }
+
+  @Test public void testAppendHttp() {
+    // I've truly no idea what append of two URLs should be, yet it does something
+    assertAppendUrl(url("http://fo%20o+/ba%20r+"), file(null, "no idea what I am doing+"),
+        "http://fo%20o+/ba%20r+/no%20idea%20what%20I%20am%20doing+");
+    assertAppendUrl(url("http://fo%20o+"), file(null, "no idea what I am doing+"),
+        "http://fo%20o+/no%20idea%20what%20I%20am%20doing+");
+    assertAppendUrl(url("http://fo%20o+/ba%20r+"), url("file:no%20idea%20what%20I%20am%20doing+"),
+        "http://fo%20o+/ba%20r+/no%20idea%20what%20I%20am%20doing+");
+    assertAppendUrl(url("http://fo%20o+"), url("file:no%20idea%20what%20I%20am%20doing+"),
+        "http://fo%20o+/no%20idea%20what%20I%20am%20doing+");
+  }
+
+  private void assertAppend(Source parent, Source child, String expected) {
+    assertThat(parent + ".append(" + child + ")",
+        parent.append(child).file().toString(),
+        // This should transparently support various OS
+        is(new File(expected).toString()));
+  }
+
+  private void assertAppendUrl(Source parent, Source child, String expected) {
+    assertThat(parent + ".append(" + child + ")",
+        parent.append(child).url().toString(),
+        is(expected));
+  }
+
+  @Test public void testSpaceInUrl() {
+    String url = "file:" + ROOT_PREFIX + "dir%20name/test%20file.json";
+    final Source foo = url(url);
+    assertEquals(url + " .file().getAbsolutePath()",
+        new File(ROOT_PREFIX + "dir name/test file.json").getAbsolutePath(),
+        foo.file().getAbsolutePath());
+  }
+
+  @Test public void testSpaceInRelativeUrl() {
+    String url = "file:dir%20name/test%20file.json";
+    final Source foo = url(url);
+    assertEquals(url + " .file().getAbsolutePath()",
+        "dir name/test file.json",
+        foo.file().getPath().replace('\\', '/'));
   }
 
   @Test public void testRelative() {
-    final Source fooBar = Sources.file(null, "/foo/bar");
-    final Source foo = Sources.file(null, "/foo");
-    final Source baz = Sources.file(null, "/baz");
+    final Source fooBar = file(null, ROOT_PREFIX + "foo/bar");
+    final Source foo = file(null, ROOT_PREFIX + "foo");
+    final Source baz = file(null, ROOT_PREFIX + "baz");
     final Source bar = fooBar.relative(foo);
     assertThat(bar.file().toString(), is("bar"));
     assertThat(fooBar.relative(baz), is(fooBar));

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
----------------------------------------------------------------------
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
index 074ad2a..3a572dd 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
@@ -29,7 +29,6 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Multimap;
 
 import org.junit.Test;
@@ -138,7 +137,7 @@ public class DruidAdapterIT {
   private CalciteAssert.AssertQuery approxQuery(URL url, String sql) {
     return CalciteAssert.that()
             .enable(enabled())
-            .with(CalciteConnectionProperty.MODEL, url.getPath())
+            .withModel(url)
             .with(CalciteConnectionProperty.APPROXIMATE_DISTINCT_COUNT, true)
             .with(CalciteConnectionProperty.APPROXIMATE_TOP_N, true)
             .with(CalciteConnectionProperty.APPROXIMATE_DECIMAL, true)
@@ -149,7 +148,7 @@ public class DruidAdapterIT {
   private CalciteAssert.AssertQuery sql(String sql, URL url) {
     return CalciteAssert.that()
         .enable(enabled())
-        .with(CalciteConnectionProperty.MODEL, url.getPath())
+        .withModel(url)
         .query(sql);
   }
 
@@ -666,7 +665,7 @@ public class DruidAdapterIT {
         + "aggs=[[SUM($1)]], sort0=[1], dir0=[DESC], fetch=[3])";
     CalciteAssert.that()
         .enable(enabled())
-        .with(CalciteConnectionProperty.MODEL, FOODMART.getPath())
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.APPROXIMATE_TOP_N, approx)
         .query(sql)
         .runs()
@@ -2965,7 +2964,7 @@ public class DruidAdapterIT {
       String expectedDruidQuery) {
     CalciteAssert.that()
         .enable(enabled())
-        .with(CalciteConnectionProperty.MODEL, FOODMART.getPath())
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.APPROXIMATE_DISTINCT_COUNT, approx)
         .query(sql)
         .runs()
@@ -3266,7 +3265,7 @@ public class DruidAdapterIT {
 
     CalciteAssert.that()
         .enable(enabled())
-        .with(CalciteConnectionProperty.MODEL, WIKI_AUTO2.getPath())
+        .withModel(WIKI_AUTO2)
         .with(CalciteConnectionProperty.TIME_ZONE, "Asia/Kolkata")
         .query(sql)
         .runs()
@@ -3288,7 +3287,7 @@ public class DruidAdapterIT {
         + "\"locale\":\"und\"}}";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", WIKI_AUTO2.getPath()))
+        .withModel(WIKI_AUTO2)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "Asia/Kolkata")
         .query(sql)
         .runs()
@@ -3314,7 +3313,7 @@ public class DruidAdapterIT {
 
     CalciteAssert.that()
         .enable(enabled())
-        .with(CalciteConnectionProperty.MODEL, WIKI_AUTO2.getPath())
+        .withModel(WIKI_AUTO2)
         .with(CalciteConnectionProperty.TIME_ZONE, "Asia/Kolkata")
         .query(sql)
         .runs()
@@ -3347,7 +3346,7 @@ public class DruidAdapterIT {
         + "\"expression\",\"name\":\"vc\",\"expression\":\"timestamp_parse";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "Asia/Kolkata")
         .query(sql)
         .runs()
@@ -3617,7 +3616,7 @@ public class DruidAdapterIT {
     final String filterTimezoneName = "America/Los_Angeles";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), filterTimezoneName)
         .query(sql)
         .runs()
@@ -3634,7 +3633,7 @@ public class DruidAdapterIT {
     final String filterTimezoneName = "America/Los_Angeles";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), filterTimezoneName)
         .query(sql)
         .runs()
@@ -3649,7 +3648,7 @@ public class DruidAdapterIT {
             + "group by EXTRACT(HOUR from \"timestamp\") ";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "America/Los_Angeles")
         .query(sql)
         .runs()
@@ -3661,7 +3660,7 @@ public class DruidAdapterIT {
             + "group by EXTRACT(HOUR from \"timestamp\") ";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "EST")
         .query(sql2)
         .runs()
@@ -3672,7 +3671,7 @@ public class DruidAdapterIT {
             + "group by EXTRACT(HOUR from \"timestamp\") ";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "UTC")
         .query(sql3)
         .runs()
@@ -3686,7 +3685,7 @@ public class DruidAdapterIT {
         + "group by EXTRACT(HOUR from CAST(\"timestamp\" AS TIMESTAMP)) ";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "America/Los_Angeles")
         .query(sql)
         .runs()
@@ -3698,7 +3697,7 @@ public class DruidAdapterIT {
         + "group by EXTRACT(HOUR from CAST(\"timestamp\" AS TIMESTAMP)) ";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "EST")
         .query(sql2)
         .runs()
@@ -3709,7 +3708,7 @@ public class DruidAdapterIT {
         + "group by EXTRACT(HOUR from CAST(\"timestamp\" AS TIMESTAMP)) ";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "UTC")
         .query(sql3)
         .runs()

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT2.java
----------------------------------------------------------------------
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT2.java b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT2.java
index d256d81..aca3fa1 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT2.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT2.java
@@ -29,7 +29,6 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Multimap;
 
 import org.junit.Test;
@@ -115,7 +114,7 @@ public class DruidAdapterIT2 {
   private CalciteAssert.AssertQuery approxQuery(URL url, String sql) {
     return CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", url.getPath()))
+        .withModel(url)
         .with(CalciteConnectionProperty.APPROXIMATE_DISTINCT_COUNT.camelName(), true)
         .with(CalciteConnectionProperty.APPROXIMATE_TOP_N.camelName(), true)
         .with(CalciteConnectionProperty.APPROXIMATE_DECIMAL.camelName(), true)
@@ -126,7 +125,7 @@ public class DruidAdapterIT2 {
   private CalciteAssert.AssertQuery sql(String sql, URL url) {
     return CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", url.getPath()))
+        .withModel(url)
         .query(sql);
   }
 
@@ -426,7 +425,7 @@ public class DruidAdapterIT2 {
         + "aggs=[[SUM($1)]], sort0=[1], dir0=[DESC], fetch=[3])";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.APPROXIMATE_TOP_N.name(), approx)
         .query(sql)
         .runs()
@@ -2664,7 +2663,7 @@ public class DruidAdapterIT2 {
                                            String expectedDruidQuery) {
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .with(CalciteConnectionProperty.APPROXIMATE_DISTINCT_COUNT.camelName(), approx)
         .query(sql)
         .runs()
@@ -2851,7 +2850,7 @@ public class DruidAdapterIT2 {
         + "\"expression\",\"name\":\"vc\",\"expression\":\"timestamp_floor(\\\"__time\\\"";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .query(sql)
         .runs()
         .queryContains(druidChecker(druidQuery))
@@ -3119,7 +3118,7 @@ public class DruidAdapterIT2 {
         + "'((timestamp_extract(\\'__time\\'";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .query(sql)
         .runs()
         .returnsOrdered("EXPR$0=86829")
@@ -3134,7 +3133,7 @@ public class DruidAdapterIT2 {
         + "'((timestamp_extract(\\'__time\\'";
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .withModel(FOODMART)
         .query(sql)
         .runs()
         .returnsOrdered("EXPR$0=7033")

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index f3f5432..c4354e6 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -21,6 +21,7 @@ import org.apache.calcite.adapter.csv.CsvStreamTableFactory;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableMap;
@@ -33,9 +34,6 @@ import org.junit.Test;
 import java.io.File;
 import java.io.PrintStream;
 import java.io.PrintWriter;
-import java.io.UnsupportedEncodingException;
-import java.net.URL;
-import java.net.URLDecoder;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -367,17 +365,7 @@ public class CsvTest {
   }
 
   private String resourcePath(String path) {
-    final URL url = CsvTest.class.getResource("/" + path);
-    // URL converts a space to %20, undo that.
-    try {
-      String s = URLDecoder.decode(url.toString(), "UTF-8");
-      if (s.startsWith("file:")) {
-        s = s.substring("file:".length());
-      }
-      return s;
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException(e);
-    }
+    return Sources.of(CsvTest.class.getResource("/" + path)).file().getAbsolutePath();
   }
 
   private static void collect(List<String> result, ResultSet resultSet)

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/file/src/main/java/org/apache/calcite/adapter/file/FileReader.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileReader.java b/file/src/main/java/org/apache/calcite/adapter/file/FileReader.java
index 6eecce4..ba092e8 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileReader.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileReader.java
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Scrapes HTML tables from URLs using Jsoup.
@@ -65,7 +66,7 @@ public class FileReader implements Iterable<Elements> {
       if (proto.equals("file")) {
         doc = Jsoup.parse(source.file(), this.charset.name());
       } else {
-        doc = Jsoup.connect(source.path()).get();
+        doc = Jsoup.parse(source.url(), (int) TimeUnit.SECONDS.toMillis(20));
       }
     } catch (IOException e) {
       throw new FileReaderException("Cannot read " + source.path(), e);

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/file/src/test/java/org/apache/calcite/adapter/file/FileReaderTest.java
----------------------------------------------------------------------
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/FileReaderTest.java b/file/src/test/java/org/apache/calcite/adapter/file/FileReaderTest.java
index 5681cb9..d4c41dc 100644
--- a/file/src/test/java/org/apache/calcite/adapter/file/FileReaderTest.java
+++ b/file/src/test/java/org/apache/calcite/adapter/file/FileReaderTest.java
@@ -35,7 +35,6 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.net.MalformedURLException;
-import java.net.URL;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -66,9 +65,7 @@ public class FileReaderTest {
   }
 
   private static String resourcePath(String path) throws Exception {
-    final URL url = FileReaderTest.class.getResource("/" + path);
-    final File file = new File(url.toURI());
-    return file.getAbsolutePath();
+    return Sources.of(FileReaderTest.class.getResource("/" + path)).file().getAbsolutePath();
   }
 
   /** Tests {@link FileReader} URL instantiation - no path. */
@@ -81,7 +78,11 @@ public class FileReaderTest {
     //   sun.security.provider.certpath.SunCertPathBuilderException:
     //   unable to find valid certification path to requested target
     final String r = System.getProperty("java.runtime.name");
-    Assume.assumeTrue(!r.equals("OpenJDK Runtime Environment"));
+    // http://openjdk.java.net/jeps/319 => root certificates are bundled with JEP 10
+    Assume.assumeTrue("Java 10+ should have root certificates (JEP 319). Runtime is "
+            + r + ", Jave major version is " + TestUtil.getJavaMajorVersion(),
+        !r.equals("OpenJDK Runtime Environment")
+            || TestUtil.getJavaMajorVersion() > 10);
 
     FileReader t = new FileReader(STATES_SOURCE);
     t.refresh();
@@ -114,7 +115,7 @@ public class FileReaderTest {
   /** Tests failed {@link FileReader} instantiation - malformed URL. */
   @Test public void testFileReaderMalUrl() throws FileReaderException {
     try {
-      final Source badSource = Sources.url("bad" + CITIES_SOURCE.path());
+      final Source badSource = Sources.url("bad" + CITIES_SOURCE.url());
       fail("expected exception, got " + badSource);
     } catch (RuntimeException e) {
       assertThat(e.getCause(), instanceOf(MalformedURLException.class));

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
----------------------------------------------------------------------
diff --git a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
index 5e99119..6c9c765 100644
--- a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
+++ b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
@@ -17,11 +17,11 @@
 package org.apache.calcite.adapter.geode.rel;
 
 import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.util.Sources;
 
 import org.junit.Assert;
 
 import java.io.PrintStream;
-import java.net.URL;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -118,12 +118,7 @@ public class BaseGeodeAdapterIT {
   }
 
   private String resourcePath(String path) {
-    final URL url = GeodeAdapterIT.class.getResource("/" + path);
-    String s = url.toString();
-    if (s.startsWith("file:")) {
-      s = s.substring("file:".length());
-    }
-    return s;
+    return Sources.of(GeodeAdapterIT.class.getResource("/" + path)).file().getAbsolutePath();
   }
 
   private void output(ResultSet resultSet, PrintStream out)

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeAdapterBookshopIT.java
----------------------------------------------------------------------
diff --git a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeAdapterBookshopIT.java b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeAdapterBookshopIT.java
index 4e92f9d..cd38841 100644
--- a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeAdapterBookshopIT.java
+++ b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeAdapterBookshopIT.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.adapter.geode.rel;
 
 import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableMap;
@@ -43,8 +44,8 @@ public class GeodeAdapterBookshopIT {
    */
   public static final ImmutableMap<String, String> GEODE =
       ImmutableMap.of("model",
-          GeodeAdapterBookshopIT.class.getResource("/model-bookshop.json")
-              .getPath());
+          Sources.of(GeodeAdapterBookshopIT.class.getResource("/model-bookshop.json"))
+              .file().getAbsolutePath());
 
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeZipsIT.java
----------------------------------------------------------------------
diff --git a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeZipsIT.java b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeZipsIT.java
index f23946c..2ca7df5 100644
--- a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeZipsIT.java
+++ b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/GeodeZipsIT.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.adapter.geode.rel;
 
 import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableMap;
@@ -42,8 +43,8 @@ public class GeodeZipsIT {
    */
   public static final ImmutableMap<String, String> GEODE_ZIPS =
       ImmutableMap.of("CONFORMANCE", "LENIENT", "model",
-          GeodeZipsIT.class.getResource("/model-zips.json")
-              .getPath());
+          Sources.of(GeodeZipsIT.class.getResource("/model-zips.json"))
+              .file().getAbsolutePath());
 
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/pig/src/test/java/org/apache/calcite/test/AbstractPigTest.java
----------------------------------------------------------------------
diff --git a/pig/src/test/java/org/apache/calcite/test/AbstractPigTest.java b/pig/src/test/java/org/apache/calcite/test/AbstractPigTest.java
index 4840ce2..f2bc905 100644
--- a/pig/src/test/java/org/apache/calcite/test/AbstractPigTest.java
+++ b/pig/src/test/java/org/apache/calcite/test/AbstractPigTest.java
@@ -16,8 +16,7 @@
  */
 package org.apache.calcite.test;
 
-import java.io.File;
-import java.net.URISyntaxException;
+import org.apache.calcite.util.Sources;
 
 /**
  * Common methods inheritable by all Pig-specific test classes.
@@ -25,11 +24,7 @@ import java.net.URISyntaxException;
 public abstract class AbstractPigTest {
 
   protected String getFullPathForTestDataFile(String fileName) {
-    try {
-      return new File(getClass().getResource("/" + fileName).toURI()).getAbsolutePath();
-    } catch (URISyntaxException e) {
-      throw new RuntimeException(e);
-    }
+    return Sources.of(getClass().getResource("/" + fileName)).file().getAbsolutePath();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/pig/src/test/java/org/apache/calcite/test/PigAdapterTest.java
----------------------------------------------------------------------
diff --git a/pig/src/test/java/org/apache/calcite/test/PigAdapterTest.java b/pig/src/test/java/org/apache/calcite/test/PigAdapterTest.java
index 0bc4544..82b5de9 100644
--- a/pig/src/test/java/org/apache/calcite/test/PigAdapterTest.java
+++ b/pig/src/test/java/org/apache/calcite/test/PigAdapterTest.java
@@ -16,12 +16,12 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.util.Sources;
+
 import com.google.common.collect.ImmutableMap;
 
 import org.junit.Test;
 
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
 import java.util.List;
 import java.util.function.Consumer;
 
@@ -35,16 +35,8 @@ public class PigAdapterTest extends AbstractPigTest {
   // Undo the %20 replacement of a space by URL
   public static final ImmutableMap<String, String> MODEL =
       ImmutableMap.of("model",
-          decodeUrl(PigAdapterTest.class.getResource("/model.json").getPath()));
-
-  /** URL-decodes the given string with UTF-8 encoding */
-  private static String decodeUrl(String urlEncoded) {
-    try {
-      return URLDecoder.decode(urlEncoded, "UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException(e);
-    }
-  }
+          Sources.of(PigAdapterTest.class.getResource("/model.json"))
+              .file().getAbsolutePath());
 
   @Test public void testScanAndFilter() throws Exception {
     CalciteAssert.that()

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/plus/src/test/java/org/apache/calcite/adapter/os/OsAdapterTest.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/org/apache/calcite/adapter/os/OsAdapterTest.java b/plus/src/test/java/org/apache/calcite/adapter/os/OsAdapterTest.java
index a2d7f78..7f0c053 100644
--- a/plus/src/test/java/org/apache/calcite/adapter/os/OsAdapterTest.java
+++ b/plus/src/test/java/org/apache/calcite/adapter/os/OsAdapterTest.java
@@ -22,6 +22,7 @@ import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.test.CalciteAssert;
 import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 
 import org.hamcrest.CoreMatchers;
@@ -66,7 +67,8 @@ public class OsAdapterTest {
   /** Returns whether there is a ".git" directory in this directory or in a
    * directory between this directory and root. */
   private static boolean hasGit() {
-    final String path = OsAdapterTest.class.getResource("/").getPath();
+    final String path = Sources.of(OsAdapterTest.class.getResource("/"))
+        .file().getAbsolutePath();
     File f = new File(path);
     for (;;) {
       if (f == null || !f.exists()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/2f33a0c5/src/main/config/forbidden-apis/signatures.txt
----------------------------------------------------------------------
diff --git a/src/main/config/forbidden-apis/signatures.txt b/src/main/config/forbidden-apis/signatures.txt
index a415c28..343fa38 100644
--- a/src/main/config/forbidden-apis/signatures.txt
+++ b/src/main/config/forbidden-apis/signatures.txt
@@ -51,6 +51,9 @@ java.lang.Runtime#exec(java.lang.String[], java.lang.String[], java.io.File)
 @defaultMessage For an enum, use == rather than equals
 java.lang.Enum#equals(java.lang.Object)
 
+@defaultMessage It does not handle encoded URLs, use Sources.of(URL).file() instead
+java.net.URL#getPath()
+
 # Preconditions.checkArgument,
 # Preconditions.checkPositionIndex, and
 # Preconditions.checkState are still OK