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 2017/02/23 16:50:52 UTC

[3/6] calcite git commit: Following [CALCITE-884] Add CSV, JSON functionality to file adapter

Following [CALCITE-884] Add CSV, JSON functionality to file adapter

Add interface Source, as a common facade over File and URL;
change example CSV adapter to use Source rather than File.

Relative paths in CSV and File adapter are now relative to the model,
rather than relative to the current directory. This makes it easier to
use the same model.json file from tests and from sqlline.

Create table "States_as_of", a snapshot the "States" wikipedia table
at a particular date, to ensure test stability.


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

Branch: refs/heads/master
Commit: 24c25fc30315f308df13eb303177163e5a972949
Parents: cf46d3b
Author: Julian Hyde <jh...@apache.org>
Authored: Sun Apr 17 23:51:14 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Feb 22 21:09:48 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/util/Source.java    |  72 +++++++
 .../java/org/apache/calcite/util/Sources.java   | 205 +++++++++++++++++++
 .../org/apache/calcite/test/CalciteSuite.java   |   2 +
 .../org/apache/calcite/util/SourceTest.java     |  45 ++++
 .../calcite/adapter/csv/CsvEnumerator.java      |  44 ++--
 .../calcite/adapter/csv/CsvFilterableTable.java |   8 +-
 .../calcite/adapter/csv/CsvScannableTable.java  |  10 +-
 .../apache/calcite/adapter/csv/CsvSchema.java   |  32 +--
 .../calcite/adapter/csv/CsvStreamReader.java    |  15 +-
 .../adapter/csv/CsvStreamScannableTable.java    |  12 +-
 .../adapter/csv/CsvStreamTableFactory.java      |   5 +-
 .../apache/calcite/adapter/csv/CsvTable.java    |  17 +-
 .../calcite/adapter/csv/CsvTableFactory.java    |  11 +-
 .../adapter/csv/CsvTranslatableTable.java       |   8 +-
 .../calcite/adapter/csv/JsonEnumerator.java     |  14 +-
 .../apache/calcite/adapter/csv/JsonTable.java   |  11 +-
 file/pom.xml                                    |  42 ++--
 .../calcite/adapter/file/FileEnumerator.java    |  16 +-
 .../apache/calcite/adapter/file/FileReader.java |  52 ++---
 .../calcite/adapter/file/FileRowConverter.java  |  24 ++-
 .../apache/calcite/adapter/file/FileSchema.java |  59 +++++-
 .../calcite/adapter/file/FileSchemaFactory.java |   7 +-
 .../apache/calcite/adapter/file/FileTable.java  |  23 ++-
 .../calcite/adapter/file/FileReaderTest.java    | 162 +++++++--------
 .../apache/calcite/adapter/file/SqlTest.java    | 114 +++++------
 file/src/test/resources/sales.json              |  20 +-
 file/src/test/resources/testModel.json          |  22 +-
 file/src/test/resources/wiki.json               |  41 +++-
 pom.xml                                         |   8 +-
 29 files changed, 760 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/core/src/main/java/org/apache/calcite/util/Source.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Source.java b/core/src/main/java/org/apache/calcite/util/Source.java
new file mode 100644
index 0000000..f92a7de
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/Source.java
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.net.URL;
+
+/**
+ * Source of data.
+ */
+public interface Source {
+  URL url();
+  File file();
+  String path();
+  Reader reader() throws IOException;
+  InputStream openStream() throws IOException;
+  String protocol();
+
+  /** Looks for a suffix on a path and returns
+   * either the path with the suffix removed
+   * or the original path. */
+  Source trim(String suffix);
+
+  /** Looks for a suffix on a path and returns
+   * either the path with the suffix removed
+   * or null. */
+  Source trimOrNull(String suffix);
+
+  /** Returns a source whose path concatenates this with a child.
+   *
+   * <p>For example,
+   * <ul>
+   *   <li>source("/foo").append(source("bar"))
+   *   returns source("/foo/bar")
+   *   <li>source("/foo").append(source("/bar"))
+   *   returns source("/bar")
+   *   because "/bar" was already absolute
+   * </ul>
+   */
+  Source append(Source child);
+
+  /** Returns a relative source, if this source is a child of a given base.
+   *
+   * <p>For example,
+   * <ul>
+   *   <li>source("/foo/bar").relative(source("/foo"))
+   *   returns source("bar")
+   *   <li>source("/baz/bar").relative(source("/foo"))
+   *   returns source("/baz/bar")
+   * </ul>
+   */
+  Source relative(Source source);
+}
+
+// End Source.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/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
new file mode 100644
index 0000000..946102f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/Sources.java
@@ -0,0 +1,205 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.zip.GZIPInputStream;
+
+/**
+ * Utilities for {@link Source}.
+ */
+public abstract class Sources {
+  private Sources() {}
+
+  public static Source of(File file) {
+    return new MySource(file);
+  }
+
+  public static Source file(File baseDirectory, String fileName) {
+    final File file = new File(fileName);
+    if (baseDirectory != null && !file.isAbsolute()) {
+      return of(new File(baseDirectory, fileName));
+    } else {
+      return of(file);
+    }
+  }
+
+  public static Source url(String url) {
+    try {
+      URL url_ = new URL(url);
+      return new MySource(url_);
+    } catch (MalformedURLException e) {
+      throw new RuntimeException("Malformed URL: '" + url + "'", e);
+    }
+  }
+
+  private static String trim(String s, String suffix) {
+    String trimmed = trimOrNull(s, suffix);
+    return trimmed != null ? trimmed : s;
+  }
+
+  /** Looks for a suffix on a path and returns
+   * either the path with the suffix removed
+   * or null. */
+  private static String trimOrNull(String s, String suffix) {
+    return s.endsWith(suffix)
+        ? s.substring(0, s.length() - suffix.length())
+        : null;
+  }
+
+  private static boolean isFile(Source source) {
+    return source.protocol().equals("file");
+  }
+
+  /** Implementation of {@link Source}. */
+  private static class MySource implements Source {
+    private final File file;
+    private final URL url;
+
+    private MySource(URL url) {
+      this.url = Preconditions.checkNotNull(url);
+      if (url.getProtocol().equals("file")) {
+        this.file = new File(url.getFile());
+      } else {
+        this.file = null;
+      }
+    }
+
+    private MySource(File file) {
+      this.file = Preconditions.checkNotNull(file);
+      this.url = null;
+    }
+
+    @Override public String toString() {
+      return (url != null ? url : file).toString();
+    }
+
+    public URL url() {
+      if (url == null) {
+        throw new UnsupportedOperationException();
+      }
+      return url;
+    }
+
+    public File file() {
+      if (file == null) {
+        throw new UnsupportedOperationException();
+      }
+      return file;
+    }
+
+    public String protocol() {
+      return file != null ? "file" : url.getProtocol();
+    }
+
+    public String path() {
+      return file != null ? file.getPath() : url.toExternalForm();
+    }
+
+    public Reader reader() throws IOException {
+      final InputStream is;
+      if (path().endsWith(".gz")) {
+        final InputStream fis = openStream();
+        is = new GZIPInputStream(fis);
+      } else {
+        if (file != null) {
+          return new FileReader(file);
+        }
+        is = openStream();
+      }
+      return new InputStreamReader(is);
+    }
+
+    public InputStream openStream() throws IOException {
+      if (file != null) {
+        return new FileInputStream(file);
+      } else {
+        return url.openStream();
+      }
+    }
+
+    public Source trim(String suffix) {
+      Source x = trimOrNull(suffix);
+      return x == null ? this : x;
+    }
+
+    public Source trimOrNull(String suffix) {
+      if (url != null) {
+        final String s = Sources.trimOrNull(url.toExternalForm(), suffix);
+        return s == null ? null : Sources.url(s);
+      } else {
+        final String s = Sources.trimOrNull(file.getPath(), suffix);
+        return s == null ? null : of(new File(s));
+      }
+    }
+
+    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;
+        }
+      }
+      if (url != null) {
+        return Sources.url(url + "/" + path);
+      } else {
+        return Sources.file(file, path);
+      }
+    }
+
+    public Source relative(Source parent) {
+      if (isFile(parent)) {
+        if (isFile(this)
+            && file.getPath().startsWith(parent.file().getPath())) {
+          String rest = file.getPath().substring(parent.file().getPath().length());
+          if (rest.startsWith(File.separator)) {
+            return Sources.file(null, rest.substring(File.separator.length()));
+          }
+        }
+        return this;
+      } else {
+        if (!isFile(this)) {
+          String rest = Sources.trimOrNull(url.toExternalForm(),
+              parent.url().toExternalForm());
+          if (rest != null
+              && rest.startsWith("/")) {
+            return Sources.file(null, rest.substring(1));
+          }
+        }
+        return this;
+      }
+    }
+  }
+}
+
+// End Sources.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 5fdd793..e64422d 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -56,6 +56,7 @@ import org.apache.calcite.util.PartiallyOrderedSetTest;
 import org.apache.calcite.util.PermutationTestCase;
 import org.apache.calcite.util.PrecedenceClimbingParserTest;
 import org.apache.calcite.util.ReflectVisitorTest;
+import org.apache.calcite.util.SourceTest;
 import org.apache.calcite.util.UtilTest;
 import org.apache.calcite.util.graph.DirectedGraphTest;
 import org.apache.calcite.util.mapping.MappingTest;
@@ -84,6 +85,7 @@ import org.junit.runners.Suite;
     RelCollationTest.class,
     UtilTest.class,
     PrecedenceClimbingParserTest.class,
+    SourceTest.class,
     MappingTest.class,
     CalciteResourceTest.class,
     FilteratorTest.class,

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/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
new file mode 100644
index 0000000..142543b
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/util/SourceTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+import org.junit.Test;
+
+import static org.hamcrest.core.Is.is;
+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"));
+  }
+
+  @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 bar = fooBar.relative(foo);
+    assertThat(bar.file().toString(), is("bar"));
+    assertThat(fooBar.relative(baz), is(fooBar));
+  }
+}
+
+// End SourceTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
index c43ce39..ba4ff10 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
@@ -21,16 +21,13 @@ import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Source;
 
 import org.apache.commons.lang3.time.FastDateFormat;
 
 import au.com.bytecode.opencsv.CSVReader;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileReader;
 import java.io.IOException;
-import java.io.InputStreamReader;
 import java.io.Reader;
 import java.text.ParseException;
 import java.util.ArrayList;
@@ -38,8 +35,6 @@ import java.util.Date;
 import java.util.List;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.zip.GZIPInputStream;
-
 
 /** Enumerator that reads from a CSV file.
  *
@@ -64,28 +59,28 @@ class CsvEnumerator<E> implements Enumerator<E> {
         FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt);
   }
 
-  public CsvEnumerator(File file, AtomicBoolean cancelFlag,
+  public CsvEnumerator(Source source, AtomicBoolean cancelFlag,
       List<CsvFieldType> fieldTypes) {
-    this(file, cancelFlag, fieldTypes, identityList(fieldTypes.size()));
+    this(source, cancelFlag, fieldTypes, identityList(fieldTypes.size()));
   }
 
-  public CsvEnumerator(File file, AtomicBoolean cancelFlag,
+  public CsvEnumerator(Source source, AtomicBoolean cancelFlag,
       List<CsvFieldType> fieldTypes, int[] fields) {
     //noinspection unchecked
-    this(file, cancelFlag, false, null,
+    this(source, cancelFlag, false, null,
         (RowConverter<E>) converter(fieldTypes, fields));
   }
 
-  public CsvEnumerator(File file, AtomicBoolean cancelFlag, boolean stream,
+  public CsvEnumerator(Source source, AtomicBoolean cancelFlag, boolean stream,
       String[] filterValues, RowConverter<E> rowConverter) {
     this.cancelFlag = cancelFlag;
     this.rowConverter = rowConverter;
     this.filterValues = filterValues;
     try {
       if (stream) {
-        this.reader = new CsvStreamReader(file);
+        this.reader = new CsvStreamReader(source);
       } else {
-        this.reader = openCsv(file);
+        this.reader = openCsv(source);
       }
       this.reader.readNext(); // skip header row
     } catch (IOException e) {
@@ -103,14 +98,16 @@ class CsvEnumerator<E> implements Enumerator<E> {
     }
   }
 
-  static RelDataType deduceRowType(JavaTypeFactory typeFactory, File file,
+  /** Deduces the names and types of a table's columns by reading the first line
+   * of a CSV file. */
+  static RelDataType deduceRowType(JavaTypeFactory typeFactory, Source source,
       List<CsvFieldType> fieldTypes) {
-    return deduceRowType(typeFactory, file, fieldTypes, false);
+    return deduceRowType(typeFactory, source, fieldTypes, false);
   }
 
   /** Deduces the names and types of a table's columns by reading the first line
   * of a CSV file. */
-  static RelDataType deduceRowType(JavaTypeFactory typeFactory, File file,
+  static RelDataType deduceRowType(JavaTypeFactory typeFactory, Source source,
       List<CsvFieldType> fieldTypes, Boolean stream) {
     final List<RelDataType> types = new ArrayList<>();
     final List<String> names = new ArrayList<>();
@@ -120,7 +117,7 @@ class CsvEnumerator<E> implements Enumerator<E> {
       types.add(typeFactory.createSqlType(SqlTypeName.TIMESTAMP));
     }
     try {
-      reader = openCsv(file);
+      reader = openCsv(source);
       final String[] strings = reader.readNext();
       for (String string : strings) {
         final String name;
@@ -132,7 +129,7 @@ class CsvEnumerator<E> implements Enumerator<E> {
           fieldType = CsvFieldType.of(typeString);
           if (fieldType == null) {
             System.out.println("WARNING: Found unknown type: "
-              + typeString + " in file: " + file.getAbsolutePath()
+              + typeString + " in file: " + source.path()
               + " for column: " + name
               + ". Will assume the type of column is string");
           }
@@ -170,15 +167,8 @@ class CsvEnumerator<E> implements Enumerator<E> {
     return typeFactory.createStructType(Pair.zip(names, types));
   }
 
-  public static CSVReader openCsv(File file) throws IOException {
-    final Reader fileReader;
-    if (file.getName().endsWith(".gz")) {
-      final GZIPInputStream inputStream =
-          new GZIPInputStream(new FileInputStream(file));
-      fileReader = new InputStreamReader(inputStream);
-    } else {
-      fileReader = new FileReader(file);
-    }
+  public static CSVReader openCsv(Source source) throws IOException {
+    final Reader fileReader = source.reader();
     return new CSVReader(fileReader);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
index 593de5a..7adfdfa 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
@@ -27,8 +27,8 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.FilterableTable;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Source;
 
-import java.io.File;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -42,8 +42,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 public class CsvFilterableTable extends CsvTable
     implements FilterableTable {
   /** Creates a CsvFilterableTable. */
-  CsvFilterableTable(File file, RelProtoDataType protoRowType) {
-    super(file, protoRowType);
+  public CsvFilterableTable(Source source, RelProtoDataType protoRowType) {
+    super(source, protoRowType);
   }
 
   public String toString() {
@@ -62,7 +62,7 @@ public class CsvFilterableTable extends CsvTable
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
-        return new CsvEnumerator<>(file, cancelFlag, false, filterValues,
+        return new CsvEnumerator<>(source, cancelFlag, false, filterValues,
             new CsvEnumerator.ArrayRowConverter(fieldTypes, fields));
       }
     };

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
index 5078afe..555d917 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
@@ -22,8 +22,8 @@ import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.util.Source;
 
-import java.io.File;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -35,8 +35,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 public class CsvScannableTable extends CsvTable
     implements ScannableTable {
   /** Creates a CsvScannableTable. */
-  CsvScannableTable(File file, RelProtoDataType protoRowType) {
-    super(file, protoRowType);
+  CsvScannableTable(Source source, RelProtoDataType protoRowType) {
+    super(source, protoRowType);
   }
 
   public String toString() {
@@ -48,8 +48,8 @@ public class CsvScannableTable extends CsvTable
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
-        return new CsvEnumerator<>(file, cancelFlag, false,
-            null, new CsvEnumerator.ArrayRowConverter(fieldTypes, fields));
+        return new CsvEnumerator<>(source, cancelFlag, false, null,
+            new CsvEnumerator.ArrayRowConverter(fieldTypes, fields));
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
index 521987b..27e735d 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
@@ -18,6 +18,8 @@ package org.apache.calcite.adapter.csv;
 
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -30,7 +32,7 @@ import java.util.Map;
  * is a CSV file in that directory.
  */
 public class CsvSchema extends AbstractSchema {
-  final File directoryFile;
+  private final File directoryFile;
   private final CsvTable.Flavor flavor;
 
   /**
@@ -66,6 +68,7 @@ public class CsvSchema extends AbstractSchema {
   @Override protected Map<String, Table> getTableMap() {
     // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
     // ".json.gz".
+    final Source baseSource = Sources.of(directoryFile);
     File[] files = directoryFile.listFiles(
         new FilenameFilter() {
           public boolean accept(File dir, String name) {
@@ -81,32 +84,33 @@ public class CsvSchema extends AbstractSchema {
     // Build a map from table name to table; each file becomes a table.
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
     for (File file : files) {
-      String tableName = trim(file.getName(), ".gz");
-      final String tableNameSansJson = trimOrNull(tableName, ".json");
-      if (tableNameSansJson != null) {
-        JsonTable table = new JsonTable(file);
-        builder.put(tableNameSansJson, table);
+      Source source = Sources.of(file);
+      Source sourceSansGz = source.trim(".gz");
+      final Source sourceSansJson = sourceSansGz.trimOrNull(".json");
+      if (sourceSansJson != null) {
+        JsonTable table = new JsonTable(source);
+        builder.put(sourceSansJson.relative(baseSource).path(), table);
         continue;
       }
-      tableName = trim(tableName, ".csv");
+      final Source sourceSansCsv = sourceSansGz.trim(".csv");
 
-      final Table table = createTable(file);
-      builder.put(tableName, table);
+      final Table table = createTable(source);
+      builder.put(sourceSansCsv.relative(baseSource).path(), table);
     }
     return builder.build();
   }
 
   /** Creates different sub-type of table based on the "flavor" attribute. */
-  private Table createTable(File file) {
+  private Table createTable(Source source) {
     switch (flavor) {
     case TRANSLATABLE:
-      return new CsvTranslatableTable(file, null);
+      return new CsvTranslatableTable(source, null);
     case SCANNABLE:
-      return new CsvScannableTable(file, null);
+      return new CsvScannableTable(source, null);
     case FILTERABLE:
-      return new CsvFilterableTable(file, null);
+      return new CsvFilterableTable(source, null);
     default:
-      throw new AssertionError("Unknown flavor " + flavor);
+      throw new AssertionError("Unknown flavor " + this.flavor);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java
index 27db047..9f24d4f 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.adapter.csv;
 
+import org.apache.calcite.util.Source;
+
 import org.apache.commons.io.input.Tailer;
 import org.apache.commons.io.input.TailerListener;
 import org.apache.commons.io.input.TailerListenerAdapter;
@@ -24,7 +26,6 @@ import au.com.bytecode.opencsv.CSVParser;
 import au.com.bytecode.opencsv.CSVReader;
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
 import java.util.ArrayDeque;
@@ -49,8 +50,8 @@ class CsvStreamReader extends CSVReader implements Closeable {
    */
   public static final long DEFAULT_MONITOR_DELAY = 2000;
 
-  CsvStreamReader(File csvFile) {
-    this(csvFile,
+  CsvStreamReader(Source source) {
+    this(source,
       CSVParser.DEFAULT_SEPARATOR,
       CSVParser.DEFAULT_QUOTE_CHARACTER,
       CSVParser.DEFAULT_ESCAPE_CHARACTER,
@@ -62,7 +63,7 @@ class CsvStreamReader extends CSVReader implements Closeable {
   /**
    * Creates a CsvStreamReader with supplied separator and quote char.
    *
-   * @param csvFile The file to an underlying CSV source.
+   * @param source The file to an underlying CSV source
    * @param separator The delimiter to use for separating entries
    * @param quoteChar The character to use for quoted elements
    * @param escape The character to use for escaping a separator or quote
@@ -71,14 +72,14 @@ class CsvStreamReader extends CSVReader implements Closeable {
    * @param ignoreLeadingWhiteSpace If true, parser should ignore
    *  white space before a quote in a field
    */
-  private CsvStreamReader(File csvFile, char separator, char quoteChar,
+  private CsvStreamReader(Source source, char separator, char quoteChar,
       char escape, int line, boolean strictQuotes,
       boolean ignoreLeadingWhiteSpace) {
     super(new StringReader("")); // dummy call to base constructor
     contentQueue = new ArrayDeque<>();
     TailerListener listener = new CsvContentListener(contentQueue);
-    tailer = Tailer.create(csvFile, listener, DEFAULT_MONITOR_DELAY, false,
-        true, 4096);
+    tailer = Tailer.create(source.file(), listener, DEFAULT_MONITOR_DELAY,
+        false, true, 4096);
     this.parser = new CSVParser(separator, quoteChar, escape, strictQuotes,
         ignoreLeadingWhiteSpace);
     this.skipLines = line;

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
index 72af281..df8ba11 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
@@ -27,8 +27,8 @@ import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.StreamableTable;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.util.Source;
 
-import java.io.File;
 import java.util.ArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -41,8 +41,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 public class CsvStreamScannableTable extends CsvScannableTable
     implements StreamableTable {
   /** Creates a CsvScannableTable. */
-  CsvStreamScannableTable(File file, RelProtoDataType protoRowType) {
-    super(file, protoRowType);
+  CsvStreamScannableTable(Source source, RelProtoDataType protoRowType) {
+    super(source, protoRowType);
   }
 
   public RelDataType getRowType(RelDataTypeFactory typeFactory) {
@@ -51,9 +51,9 @@ public class CsvStreamScannableTable extends CsvScannableTable
     }
     if (fieldTypes == null) {
       fieldTypes = new ArrayList<>();
-      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, file, fieldTypes, true);
+      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, source, fieldTypes, true);
     } else {
-      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, file, null, true);
+      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, source, null, true);
     }
   }
 
@@ -66,7 +66,7 @@ public class CsvStreamScannableTable extends CsvScannableTable
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
-        return new CsvEnumerator<>(file, cancelFlag, true, null,
+        return new CsvEnumerator<>(source, cancelFlag, true, null,
             new CsvEnumerator.ArrayRowConverter(fieldTypes, fields, true));
       }
     };

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamTableFactory.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamTableFactory.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamTableFactory.java
index 72d26a9..159b61d 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamTableFactory.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamTableFactory.java
@@ -22,6 +22,8 @@ import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.TableFactory;
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
 
 import java.io.File;
 import java.util.Map;
@@ -47,9 +49,10 @@ public class CsvStreamTableFactory implements TableFactory<CsvTable> {
     if (base != null && !file.isAbsolute()) {
       file = new File(base, fileName);
     }
+    final Source source = Sources.of(file);
     final RelProtoDataType protoRowType =
         rowType != null ? RelDataTypeImpl.proto(rowType) : null;
-    return new CsvStreamScannableTable(file, protoRowType);
+    return new CsvStreamScannableTable(source, protoRowType);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
index e09863b..d3d4f94 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
@@ -21,8 +21,8 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.util.Source;
 
-import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -30,13 +30,13 @@ import java.util.List;
  * Base class for table that reads CSV files.
  */
 public abstract class CsvTable extends AbstractTable {
-  protected final File file;
+  protected final Source source;
   protected final RelProtoDataType protoRowType;
   protected List<CsvFieldType> fieldTypes;
 
-  /** Creates a CsvAbstractTable. */
-  CsvTable(File file, RelProtoDataType protoRowType) {
-    this.file = file;
+  /** Creates a CsvTable. */
+  CsvTable(Source source, RelProtoDataType protoRowType) {
+    this.source = source;
     this.protoRowType = protoRowType;
   }
 
@@ -45,12 +45,11 @@ public abstract class CsvTable extends AbstractTable {
       return protoRowType.apply(typeFactory);
     }
     if (fieldTypes == null) {
-      fieldTypes = new ArrayList<CsvFieldType>();
-      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, file,
+      fieldTypes = new ArrayList<>();
+      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, source,
           fieldTypes);
     } else {
-      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory,
-          file,
+      return CsvEnumerator.deduceRowType((JavaTypeFactory) typeFactory, source,
           null);
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
index 9c279b9..048753e 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableFactory.java
@@ -22,6 +22,8 @@ import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.TableFactory;
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
 
 import java.io.File;
 import java.util.Map;
@@ -30,7 +32,7 @@ import java.util.Map;
  * Factory that creates a {@link CsvTranslatableTable}.
  *
  * <p>Allows a CSV table to be included in a model.json file, even in a
- * schema that is not based upon {@link CsvSchema}.</p>
+ * schema that is not based upon {@link CsvSchema}.
  */
 @SuppressWarnings("UnusedDeclaration")
 public class CsvTableFactory implements TableFactory<CsvTable> {
@@ -41,15 +43,12 @@ public class CsvTableFactory implements TableFactory<CsvTable> {
   public CsvTable create(SchemaPlus schema, String name,
       Map<String, Object> operand, RelDataType rowType) {
     String fileName = (String) operand.get("file");
-    File file = new File(fileName);
     final File base =
         (File) operand.get(ModelHandler.ExtraOperand.BASE_DIRECTORY.camelName);
-    if (base != null && !file.isAbsolute()) {
-      file = new File(base, fileName);
-    }
+    final Source source = Sources.file(base, fileName);
     final RelProtoDataType protoRowType =
         rowType != null ? RelDataTypeImpl.proto(rowType) : null;
-    return new CsvScannableTable(file, protoRowType);
+    return new CsvScannableTable(source, protoRowType);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
index 8970e28..c9424f0 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
@@ -30,8 +30,8 @@ import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.util.Source;
 
-import java.io.File;
 import java.lang.reflect.Type;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -41,8 +41,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 public class CsvTranslatableTable extends CsvTable
     implements QueryableTable, TranslatableTable {
   /** Creates a CsvTable. */
-  CsvTranslatableTable(File file, RelProtoDataType protoRowType) {
-    super(file, protoRowType);
+  CsvTranslatableTable(Source source, RelProtoDataType protoRowType) {
+    super(source, protoRowType);
   }
 
   public String toString() {
@@ -57,7 +57,7 @@ public class CsvTranslatableTable extends CsvTable
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object>() {
       public Enumerator<Object> enumerator() {
-        return new CsvEnumerator<>(file, cancelFlag, fieldTypes, fields);
+        return new CsvEnumerator<>(source, cancelFlag, fieldTypes, fields);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
index e129798..d4c6084 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
@@ -18,11 +18,11 @@ package org.apache.calcite.adapter.csv;
 
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.util.Source;
 
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.List;
 
@@ -30,14 +30,20 @@ import java.util.List;
 class JsonEnumerator implements Enumerator<Object[]> {
   private final Enumerator<Object> enumerator;
 
-  public JsonEnumerator(File file) {
+  public JsonEnumerator(Source source) {
     try {
       final ObjectMapper mapper = new ObjectMapper();
       mapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
       mapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true);
       mapper.configure(JsonParser.Feature.ALLOW_COMMENTS, true);
-      //noinspection unchecked
-      List<Object> list = mapper.readValue(file, List.class);
+      List<Object> list;
+      if (source.protocol().equals("file")) {
+        //noinspection unchecked
+        list = mapper.readValue(source.file(), List.class);
+      } else {
+        //noinspection unchecked
+        list = mapper.readValue(source.url(), List.class);
+      }
       enumerator = Linq4j.enumerator(list);
     } catch (IOException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
index 8bdeff1..3c7c202 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
@@ -25,18 +25,17 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.sql.type.SqlTypeName;
-
-import java.io.File;
+import org.apache.calcite.util.Source;
 
 /**
  * Table based on a JSON file.
  */
 public class JsonTable extends AbstractTable implements ScannableTable {
-  private final File file;
+  private final Source source;
 
   /** Creates a JsonTable. */
-  JsonTable(File file) {
-    this.file = file;
+  public JsonTable(Source source) {
+    this.source = source;
   }
 
   public String toString() {
@@ -54,7 +53,7 @@ public class JsonTable extends AbstractTable implements ScannableTable {
   public Enumerable<Object[]> scan(DataContext root) {
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
-        return new JsonEnumerator(file);
+        return new JsonEnumerator(source);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/pom.xml
----------------------------------------------------------------------
diff --git a/file/pom.xml b/file/pom.xml
index d8173a5..0490e00 100644
--- a/file/pom.xml
+++ b/file/pom.xml
@@ -36,10 +36,30 @@ limitations under the License.
 
   <dependencies>
     <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-linq4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-example-csv</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica-core</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.joestelmach</groupId>
       <artifactId>natty</artifactId>
     </dependency>
@@ -49,14 +69,6 @@ limitations under the License.
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-linq4j</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.jsoup</groupId>
       <artifactId>jsoup</artifactId>
     </dependency>
@@ -65,20 +77,8 @@ limitations under the License.
   <build>
     <plugins>
       <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-	      <artifactId>jalopy-maven-plugin</artifactId>
-	      <version>1.0-alpha-1</version>
-	      <configuration>
-	        <srcExcludesPattern>**/CsvTableScan.java</srcExcludesPattern>
-            <failOnError>false</failOnError>
-            <convention>${top.dir}src/main/config/jalopy.xml</convention>
-	      </configuration>
-      </plugin>
-
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.18</version>
         <configuration>
           <includes>
             <include>org/apache/calcite/adapter/file/FileSuite.java</include>
@@ -89,7 +89,7 @@ limitations under the License.
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
-        <version>2.8</version>
+        <version>${maven-dependency-plugin.version}</version>
         <executions>
           <execution>
             <id>copy-dependencies</id>

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/main/java/org/apache/calcite/adapter/file/FileEnumerator.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileEnumerator.java b/file/src/main/java/org/apache/calcite/adapter/file/FileEnumerator.java
index 952ccd3..d563910 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileEnumerator.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileEnumerator.java
@@ -23,24 +23,20 @@ import org.jsoup.select.Elements;
 import java.util.Iterator;
 
 /**
- * Wraps WebReader and FileRowConverter, enumerates tr Elements as
- * table rows.
+ * Wraps {@link FileReader} and {@link FileRowConverter}, enumerates tr DOM
+ * elements as table rows.
  */
 class FileEnumerator implements Enumerator<Object> {
-
   private final Iterator<Elements> iterator;
   private final FileRowConverter converter;
   private final int[] fields;
   private Object current;
 
-  public FileEnumerator(Iterator<Elements> iterator,
-      FileRowConverter converter) {
-    this.iterator = iterator;
-    this.converter = converter;
-    this.fields = identityList(this.converter.width());
+  FileEnumerator(Iterator<Elements> iterator, FileRowConverter converter) {
+    this(iterator, converter, identityList(converter.width()));
   }
 
-  public FileEnumerator(Iterator<Elements> iterator, FileRowConverter converter,
+  FileEnumerator(Iterator<Elements> iterator, FileRowConverter converter,
       int[] fields) {
     this.iterator = iterator;
     this.converter = converter;
@@ -64,6 +60,8 @@ class FileEnumerator implements Enumerator<Object> {
         current = null;
         return false;
       }
+    } catch (RuntimeException | Error e) {
+      throw e;
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/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 bd26f4a..fc6904b 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
@@ -16,20 +16,14 @@
  */
 package org.apache.calcite.adapter.file;
 
-import org.jsoup.Jsoup;
+import org.apache.calcite.util.Source;
 
+import org.jsoup.Jsoup;
 import org.jsoup.nodes.Document;
 import org.jsoup.nodes.Element;
-
 import org.jsoup.select.Elements;
 
-import java.io.File;
-
 import java.io.IOException;
-
-import java.net.MalformedURLException;
-import java.net.URL;
-
 import java.util.Iterator;
 
 /**
@@ -39,53 +33,46 @@ public class FileReader implements Iterable<Elements> {
 
   private static final String DEFAULT_CHARSET = "UTF-8";
 
-  private final URL url;
+  private final Source source;
   private final String selector;
   private final Integer index;
   private final String charset = DEFAULT_CHARSET;
   private Element tableElement;
   private Elements headings;
 
-  public FileReader(String url, String selector, Integer index)
+  public FileReader(Source source, String selector, Integer index)
       throws FileReaderException {
-    if (url == null) {
+    if (source == null) {
       throw new FileReaderException("URL must not be null");
     }
-
-    try {
-      this.url = new URL(url);
-    } catch (MalformedURLException e) {
-      throw new FileReaderException("Malformed URL: '" + url + "'", e);
-    }
+    this.source = source;
     this.selector = selector;
     this.index = index;
   }
 
-  public FileReader(String url, String selector) throws FileReaderException {
-    this(url, selector, null);
+  public FileReader(Source source, String selector) throws FileReaderException {
+    this(source, selector, null);
   }
 
-  public FileReader(String url) throws FileReaderException {
-    this(url, null, null);
+  public FileReader(Source source) throws FileReaderException {
+    this(source, null, null);
   }
 
   private void getTable() throws FileReaderException {
-
-    Document doc;
+    final Document doc;
     try {
-      String proto = this.url.getProtocol();
+      String proto = source.protocol();
       if (proto.equals("file")) {
-        doc = Jsoup.parse(new File(this.url.getFile()), this.charset);
+        doc = Jsoup.parse(source.file(), this.charset);
       } else {
-        doc = Jsoup.connect(this.url.toString()).get();
+        doc = Jsoup.connect(source.path()).get();
       }
     } catch (IOException e) {
-      throw new FileReaderException("Cannot read " + this.url.toString(), e);
+      throw new FileReaderException("Cannot read " + source.path(), e);
     }
 
     this.tableElement = (this.selector != null && !this.selector.equals(""))
         ? getSelectedTable(doc, this.selector) : getBestTable(doc);
-
   }
 
   private Element getSelectedTable(Document doc, String selector)
@@ -153,15 +140,16 @@ public class FileReader implements Iterable<Elements> {
   }
 
   private String tableKey() {
-    return "Table: {url: " + this.url + ", selector: " + this.selector;
+    return "Table: {url: " + this.source + ", selector: " + this.selector + "}";
   }
 
   public FileReaderIterator iterator() {
     if (this.tableElement == null) {
       try {
         getTable();
+      } catch (RuntimeException | Error e) {
+        throw e;
       } catch (Exception e) {
-        // TODO: temporary hack
         throw new RuntimeException(e);
       }
     }
@@ -201,8 +189,8 @@ public class FileReader implements Iterable<Elements> {
   }
 
   /** Iterates over HTML tables, returning an Elements per row. */
-  private class FileReaderIterator implements Iterator<Elements> {
-    Iterator<Element> rowIterator;
+  private static class FileReaderIterator implements Iterator<Elements> {
+    final Iterator<Element> rowIterator;
 
     FileReaderIterator(Elements rows) {
       this.rowIterator = rows.iterator();

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
index 4510f3b..a6c1bcc 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.util.Pair;
 
@@ -32,8 +33,8 @@ import java.text.NumberFormat;
 import java.text.ParseException;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -52,7 +53,7 @@ class FileRowConverter {
   private boolean initialized = false;
 
   // row parser configuration
-  private ArrayList<FieldDef> fields;
+  private final List<FieldDef> fields = new ArrayList<>();
 
   /** Creates a FileRowConverter. */
   FileRowConverter(FileReader fileReader,
@@ -69,11 +70,10 @@ class FileRowConverter {
       return;
     }
     try {
-      this.fields = new ArrayList<>();
       final Elements headerElements = this.fileReader.getHeadings();
 
       // create a name to index map for HTML table elements
-      final Map<String, Integer> headerMap = new HashMap<>();
+      final Map<String, Integer> headerMap = new LinkedHashMap<>();
       int i = 0;
       for (Element th : headerElements) {
         String heading = th.text();
@@ -123,19 +123,23 @@ class FileRowConverter {
               addFieldDef(name, type, fieldConfig, sourceIx);
             }
           }
+        } catch (RuntimeException e) {
+          throw e;
         } catch (Exception e) {
           throw new RuntimeException(e);
         }
       }
 
       // pick up any data elements not explicitly defined
-      for (String name : headerMap.keySet()) {
+      for (Map.Entry<String, Integer> e : headerMap.entrySet()) {
+        final String name = e.getKey();
         if (!sources.contains(name) && !colNames.contains(name)) {
-          addFieldDef(name, null, null, headerMap.get(name));
+          addFieldDef(name, null, null, e.getValue());
         }
       }
 
-      // ToDo
+    } catch (RuntimeException e) {
+      throw e;
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -309,9 +313,9 @@ class FileRowConverter {
     }
 
     private java.util.Date parseDate(String string) {
-      Parser parser = new Parser();
-      List groups = parser.parse(string);
-      DateGroup group = (DateGroup) groups.get(0);
+      Parser parser = new Parser(DateTimeUtils.GMT_ZONE);
+      List<DateGroup> groups = parser.parse(string);
+      DateGroup group = groups.get(0);
       return group.getDates().get(0);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java b/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
index 6df6171..388f46b 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
@@ -16,13 +16,20 @@
  */
 package org.apache.calcite.adapter.file;
 
+import org.apache.calcite.adapter.csv.CsvFilterableTable;
+import org.apache.calcite.adapter.csv.JsonTable;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.io.File;
+import java.net.MalformedURLException;
 import java.util.List;
 import java.util.Map;
 
@@ -31,18 +38,21 @@ import java.util.Map;
  * is an HTML table on a URL.
  */
 class FileSchema extends AbstractSchema {
-  private ImmutableList<Map<String, Object>> tables;
+  private final ImmutableList<Map<String, Object>> tables;
+  private final File baseDirectory;
 
   /**
    * Creates an HTML tables schema.
    *
    * @param parentSchema Parent schema
    * @param name Schema name
+   * @param baseDirectory Base directory to look for relative files, or null
    * @param tables List containing HTML table identifiers
    */
-  FileSchema(SchemaPlus parentSchema, String name,
+  FileSchema(SchemaPlus parentSchema, String name, File baseDirectory,
       List<Map<String, Object>> tables) {
     this.tables = ImmutableList.copyOf(tables);
+    this.baseDirectory = baseDirectory;
   }
 
   @Override protected Map<String, Table> getTableMap() {
@@ -50,18 +60,51 @@ class FileSchema extends AbstractSchema {
 
     for (Map<String, Object> tableDef : this.tables) {
       String tableName = (String) tableDef.get("name");
-
       try {
-        FileTable table = new FileTable(tableDef, null);
-        builder.put(tableName, table);
-      } catch (Exception e) {
-        e.printStackTrace();
-        System.out.println("Unable to instantiate table for: " + tableName);
+        addTable(builder, tableDef);
+      } catch (MalformedURLException e) {
+        throw new RuntimeException("Unable to instantiate table for: "
+            + tableName);
       }
     }
 
     return builder.build();
   }
+
+  private void addTable(ImmutableMap.Builder<String, Table> builder,
+      Map<String, Object> tableDef) throws MalformedURLException {
+    final String tableName = (String) tableDef.get("name");
+    final String url = (String) tableDef.get("url");
+    final Source source0 = Sources.url(url);
+    final Source source;
+    if (baseDirectory == null) {
+      source = source0;
+    } else {
+      source = Sources.of(baseDirectory).append(source0);
+    }
+
+    final Source sourceSansGz = source.trim(".gz");
+    final Source sourceSansJson = sourceSansGz.trimOrNull(".json");
+    if (sourceSansJson != null) {
+      JsonTable table = new JsonTable(source);
+      builder.put(Util.first(tableName, sourceSansJson.path()), table);
+      return;
+    }
+    final Source sourceSansCsv = sourceSansGz.trimOrNull(".csv");
+    if (sourceSansCsv != null) {
+      final Table table = new CsvFilterableTable(source, null);
+      builder.put(Util.first(tableName, sourceSansCsv.path()), table);
+      return;
+    }
+
+    try {
+      FileTable table = FileTable.create(source, tableDef);
+      builder.put(Util.first(tableName, source.path()), table);
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to instantiate table for: "
+          + tableName);
+    }
+  }
 }
 
 // End FileSchema.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
index 361835f..6aa0788 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
@@ -16,10 +16,12 @@
  */
 package org.apache.calcite.adapter.file;
 
+import org.apache.calcite.model.ModelHandler;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaFactory;
 import org.apache.calcite.schema.SchemaPlus;
 
+import java.io.File;
 import java.util.List;
 import java.util.Map;
 
@@ -39,8 +41,9 @@ public class FileSchemaFactory implements SchemaFactory {
       Map<String, Object> operand) {
     @SuppressWarnings("unchecked") List<Map<String, Object>> tables =
         (List) operand.get("tables");
-
-    return new FileSchema(parentSchema, name, tables);
+    final File baseDirectory =
+        (File) operand.get(ModelHandler.ExtraOperand.BASE_DIRECTORY.camelName);
+    return new FileSchema(parentSchema, name, baseDirectory, tables);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/main/java/org/apache/calcite/adapter/file/FileTable.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileTable.java b/file/src/main/java/org/apache/calcite/adapter/file/FileTable.java
index c8d1a4b..8cc77ab 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileTable.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileTable.java
@@ -35,8 +35,9 @@ import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.util.Source;
 
-import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -50,20 +51,24 @@ class FileTable extends AbstractQueryableTable
   private FileRowConverter converter;
 
   /** Creates a FileTable. */
-  FileTable(Map<String, Object> tableDef, RelProtoDataType protoRowType)
+  private FileTable(Source source, String selector, Integer index,
+      RelProtoDataType protoRowType, List<Map<String, Object>> fieldConfigs)
       throws Exception {
     super(Object[].class);
 
     this.protoRowType = protoRowType;
-    @SuppressWarnings("unchecked") ArrayList<Map<String, Object>> fieldConfigs =
-        (ArrayList<Map<String, Object>>) tableDef.get("fields");
-    String url = (String) tableDef.get("url");
-    String selector = (String) tableDef.get("selector");
-    Integer index = (Integer) tableDef.get("index");
-    this.reader = new FileReader(url, selector, index);
+    this.reader = new FileReader(source, selector, index);
     this.converter = new FileRowConverter(this.reader, fieldConfigs);
-    //System.out.println("Created FileTable: " + (String) tableDef.get("name"));
+  }
 
+  /** Creates a FileTable. */
+  static FileTable create(Source source, Map<String, Object> tableDef)
+      throws Exception {
+    @SuppressWarnings("unchecked") List<Map<String, Object>> fieldConfigs =
+        (List<Map<String, Object>>) tableDef.get("fields");
+    String selector = (String) tableDef.get("selector");
+    Integer index = (Integer) tableDef.get("index");
+    return new FileTable(source, selector, index, null, fieldConfigs);
   }
 
   public String toString() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/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 535011d..d18c1a9 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
@@ -16,14 +16,23 @@
  */
 package org.apache.calcite.adapter.file;
 
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
+
 import org.jsoup.select.Elements;
 
 import org.junit.Assume;
 import org.junit.Test;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
+import java.io.File;
+import java.net.MalformedURLException;
 import java.util.Iterator;
 
 /**
@@ -32,96 +41,95 @@ import java.util.Iterator;
 
 public class FileReaderTest {
 
-  static final String CITIES_URI =
-      "http://en.wikipedia.org/wiki/List_of_United_States_cities_by_population";
+  private static final Source CITIES_SOURCE =
+      Sources.url("http://en.wikipedia.org/wiki/List_of_United_States_cities_by_population");
 
-  static final String STATES_URI =
-      "http://en.wikipedia.org/wiki/List_of_states_and_territories_of_the_United_States";
+  private static final Source STATES_SOURCE =
+      Sources.url(
+          "http://en.wikipedia.org/wiki/List_of_states_and_territories_of_the_United_States");
 
-  /**
-   * Test FileReader URL instantiation - no path
-   */
-  @Test
-  public void testFileReaderURLNoPath() throws FileReaderException {
+  /** Converts a path that is relative to the module into a path that is
+   * relative to where the test is running. */
+  public static String file(String s) {
+    if (new File("file").exists()) {
+      return "file/" + s;
+    } else {
+      return s;
+    }
+  }
+
+  /** Tests {@link FileReader} URL instantiation - no path. */
+  @Test public void testFileReaderUrlNoPath() throws FileReaderException {
     Assume.assumeTrue(FileSuite.hazNetwork());
-    FileReader t = new FileReader(STATES_URI);
+    FileReader t = new FileReader(STATES_SOURCE);
     t.refresh();
   }
 
-  /**
-   * Test FileReader URL instantiation - with path
-   */
-  @Test
-  public void testFileReaderURLWithPath() throws FileReaderException {
+  /** Tests {@link FileReader} URL instantiation - with path. */
+  @Test public void testFileReaderUrlWithPath() throws FileReaderException {
     Assume.assumeTrue(FileSuite.hazNetwork());
     FileReader t =
-        new FileReader(CITIES_URI,
+        new FileReader(CITIES_SOURCE,
             "#mw-content-text > table.wikitable.sortable", 0);
     t.refresh();
   }
 
-  /**
-   * Test FileReader URL fetch
-   */
-  @Test
-  public void testFileReaderURLFetch() throws FileReaderException {
+  /** Tests {@link FileReader} URL fetch. */
+  @Test public void testFileReaderUrlFetch() throws FileReaderException {
     Assume.assumeTrue(FileSuite.hazNetwork());
     FileReader t =
-        new FileReader(STATES_URI,
+        new FileReader(STATES_SOURCE,
             "#mw-content-text > table.wikitable.sortable", 0);
     int i = 0;
     for (Elements row : t) {
       i++;
     }
-    assertTrue(i == 50);
+    assertThat(i, is(51));
   }
 
-  /**
-   * Test failed FileReader instantiation - malformed URL
-   */
-  @Test(expected = FileReaderException.class)
-  public void testFileReaderMalURL() throws FileReaderException {
-    FileReader t = new FileReader("bad" + CITIES_URI, "table:eq(4)");
-    t.refresh();
+  /** Tests failed {@link FileReader} instantiation - malformed URL. */
+  @Test public void testFileReaderMalUrl() throws FileReaderException {
+    try {
+      final Source badSource = Sources.url("bad" + CITIES_SOURCE.path());
+      fail("expected exception, got " + badSource);
+    } catch (RuntimeException e) {
+      assertThat(e.getCause(), instanceOf(MalformedURLException.class));
+      assertThat(e.getCause().getMessage(), is("unknown protocol: badhttp"));
+    }
   }
 
-  /**
-   * Test failed FileReader instantiation - bad URL
-   */
+  /** Tests failed {@link FileReader} instantiation - bad URL. */
   @Test(expected = FileReaderException.class)
-  public void testFileReaderBadURL() throws FileReaderException {
+  public void testFileReaderBadUrl() throws FileReaderException {
     final String uri =
         "http://ex.wikipedia.org/wiki/List_of_United_States_cities_by_population";
-    FileReader t = new FileReader(uri, "table:eq(4)");
+    FileReader t = new FileReader(Sources.url(uri), "table:eq(4)");
     t.refresh();
   }
 
-  /**
-   * Test failed FileReader instantiation - bad selector
-   */
+  /** Tests failed {@link FileReader} instantiation - bad selector. */
   @Test(expected = FileReaderException.class)
   public void testFileReaderBadSelector() throws FileReaderException {
-    FileReader t =
-        new FileReader("file:target/test-classes/tableOK.html", "table:eq(1)");
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableOK.html"));
+    FileReader t = new FileReader(source, "table:eq(1)");
     t.refresh();
   }
 
-  /**
-   * Test FileReader with static file - headings
-   */
-  @Test
-  public void testFileReaderHeadings() throws FileReaderException {
-    FileReader t = new FileReader("file:target/test-classes/tableOK.html");
+  /** Test {@link FileReader} with static file - headings. */
+  @Test public void testFileReaderHeadings() throws FileReaderException {
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableOK.html"));
+    FileReader t = new FileReader(source);
     Elements headings = t.getHeadings();
     assertTrue(headings.get(1).text().equals("H1"));
   }
 
-  /**
-   * Test FileReader with static file - data
-   */
-  @Test
-  public void testFileReaderData() throws FileReaderException {
-    FileReader t = new FileReader("file:target/test-classes/tableOK.html");
+  /** Test {@link FileReader} with static file - data. */
+  @Test public void testFileReaderData() throws FileReaderException {
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableOK.html"));
+    FileReader t = new FileReader(source);
     Iterator<Elements> i = t.iterator();
     Elements row = i.next();
     assertTrue(row.get(2).text().equals("R0C2"));
@@ -129,24 +137,20 @@ public class FileReaderTest {
     assertTrue(row.get(0).text().equals("R1C0"));
   }
 
-  /**
-   * Test FileReader with bad static file - headings
-   */
-  @Test
-  public void testFileReaderHeadingsBadFile() throws FileReaderException {
-    FileReader t =
-        new FileReader("file:target/test-classes/tableNoTheadTbody.html");
+  /** Tests {@link FileReader} with bad static file - headings. */
+  @Test public void testFileReaderHeadingsBadFile() throws FileReaderException {
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableNoTheadTbody.html"));
+    FileReader t = new FileReader(source);
     Elements headings = t.getHeadings();
     assertTrue(headings.get(1).text().equals("H1"));
   }
 
-  /**
-   * Test FileReader with bad static file - data
-   */
-  @Test
-  public void testFileReaderDataBadFile() throws FileReaderException {
-    final FileReader t =
-        new FileReader("file:target/test-classes/tableNoTheadTbody.html");
+  /** Tests {@link FileReader} with bad static file - data. */
+  @Test public void testFileReaderDataBadFile() throws FileReaderException {
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableNoTheadTbody.html"));
+    FileReader t = new FileReader(source);
     Iterator<Elements> i = t.iterator();
     Elements row = i.next();
     assertTrue(row.get(2).text().equals("R0C2"));
@@ -154,24 +158,22 @@ public class FileReaderTest {
     assertTrue(row.get(0).text().equals("R1C0"));
   }
 
-  /**
-   * Test FileReader with no headings static file - data
-   */
-  @Test
-  public void testFileReaderDataNoTH() throws FileReaderException {
-    FileReader t = new FileReader("file:target/test-classes/tableNoTH.html");
-    Iterator<Elements> i =
-        new FileReader("file:target/test-classes/tableNoTH.html").iterator();
+  /** Tests {@link FileReader} with no headings static file - data. */
+  @Test public void testFileReaderDataNoTh() throws FileReaderException {
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableNoTH.html"));
+    FileReader t = new FileReader(source);
+    Iterator<Elements> i = t.iterator();
     Elements row = i.next();
     assertTrue(row.get(2).text().equals("R0C2"));
   }
 
-  /**
-   * Test FileReader iterator with static file
-   */
-  @Test
-  public void testFileReaderIterator() throws FileReaderException {
-    FileReader t = new FileReader("file:target/test-classes/tableOK.html");
+  /** Tests {@link FileReader} iterator with static file, */
+  @Test public void testFileReaderIterator() throws FileReaderException {
+    System.out.println(new File("").getAbsolutePath());
+    final Source source =
+        Sources.file(null, file("target/test-classes/tableOK.html"));
+    FileReader t = new FileReader(source);
     Elements row = null;
     for (Elements aT : t) {
       row = aT;

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
----------------------------------------------------------------------
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java b/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
index 60549ed..86360f1 100644
--- a/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
+++ b/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
@@ -21,11 +21,9 @@ import com.google.common.base.Function;
 import org.junit.Assume;
 import org.junit.Test;
 
-import java.io.PrintStream;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Properties;
@@ -39,21 +37,13 @@ import static org.junit.Assert.assertEquals;
 public class SqlTest {
   // helper functions
 
-  private void checkSql(String model, String sql) throws SQLException {
-    checkSql(sql, model, new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          output(resultSet, System.out);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
-      }
-    });
-  }
-
-  private void checkSql(String model, String sql, final String expected)
+  private void checkSql(String model, String sql, String... expectedLines)
       throws SQLException {
+    final StringBuilder b = new StringBuilder();
+    for (String s : expectedLines) {
+      b.append(s).append('\n');
+    }
+    final String expected = b.toString();
     checkSql(sql, model, new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
@@ -78,12 +68,11 @@ public class SqlTest {
     Statement statement = null;
     try {
       Properties info = new Properties();
-      info.put("model", "target/test-classes/" + model + ".json");
+      info.put("model",
+          FileReaderTest.file("target/test-classes/" + model + ".json"));
       connection = DriverManager.getConnection("jdbc:calcite:", info);
       statement = connection.createStatement();
-      final ResultSet resultSet =
-          statement.executeQuery(
-              sql);
+      final ResultSet resultSet = statement.executeQuery(sql);
       fn.apply(resultSet);
     } finally {
       close(connection, statement);
@@ -107,23 +96,6 @@ public class SqlTest {
     return buf.toString();
   }
 
-  private void output(ResultSet resultSet, PrintStream out)
-      throws SQLException {
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    final int columnCount = metaData.getColumnCount();
-    while (resultSet.next()) {
-      for (int i = 1;; i++) {
-        out.print(resultSet.getString(i));
-        if (i < columnCount) {
-          out.print(", ");
-        } else {
-          out.println();
-          break;
-        }
-      }
-    }
-  }
-
   private void close(Connection connection, Statement statement) {
     if (statement != null) {
       try {
@@ -143,45 +115,53 @@ public class SqlTest {
 
   // tests
 
-  /**
-   * Reads from a local file and checks the result
-   */
-  @Test
-  public void testFileSelect() throws SQLException {
-    checkSql("testModel", "select H1 from T1 where H0 = 'R1C0'", "H1=R1C1\n");
+  /** Reads from a local file and checks the result. */
+  @Test public void testFileSelect() throws SQLException {
+    final String sql = "select H1 from T1 where H0 = 'R1C0'";
+    checkSql("testModel", sql, "H1=R1C1");
   }
 
-  /**
-   * Reads from a local file without table headers <TH> and checks the result
-   */
-  @Test
-  public void testNoTHSelect() throws SQLException {
+  /** Reads from a local file without table headers &lt;TH&gt; and checks the
+   * result. */
+  @Test public void testNoThSelect() throws SQLException {
     Assume.assumeTrue(FileSuite.hazNetwork());
-    checkSql("testModel",
-        "select \"col1\" from T1_NO_TH where \"col0\" like 'R0%'",
-        "col1=R0C1\n");
+    final String sql = "select \"col1\" from T1_NO_TH where \"col0\" like 'R0%'";
+    checkSql("testModel", sql, "col1=R0C1");
   }
 
-  /**
-   * Reads from a local file - finds larger table even without <TH> elements
-   */
-  @Test
-  public void testFindBiggerNoTH() throws SQLException {
-    checkSql("testModel",
-        "select \"col4\" from TABLEX2 where \"col0\" like 'R1%'",
-        "col4=R1C4\n");
+  /** Reads from a local file - finds larger table even without &lt;TH&gt;
+   * elements. */
+  @Test public void testFindBiggerNoTh() throws SQLException {
+    final String sql = "select \"col4\" from TABLEX2 where \"col0\" like 'R1%'";
+    checkSql("testModel", sql, "col4=R1C4");
   }
 
-  /**
-   * Reads from a URL and checks the result
-   */
-  @Test
-  public void testURLSelect() throws SQLException {
+  /** Reads from a URL and checks the result. */
+  @Test public void testUrlSelect() throws SQLException {
     Assume.assumeTrue(FileSuite.hazNetwork());
-    final String sql = "select \"State\", \"Statehood\" from \"States\"\n"
+    final String sql = "select \"State\", \"Statehood\" from \"States_as_of\"\n"
         + "where \"State\" = 'California'";
-    checkSql("wiki", sql,
-        "State=California; Statehood=1850-09-09\n");
+    checkSql("wiki", sql, "State=California; Statehood=1850-09-09");
+  }
+
+  /** Reads the EMPS table. */
+  @Test public void testSalesEmps() throws SQLException {
+    final String sql = "select * from sales.emps";
+    checkSql("sales", sql,
+        "EMPNO=100; NAME=Fred; DEPTNO=30",
+        "EMPNO=110; NAME=Eric; DEPTNO=20",
+        "EMPNO=110; NAME=John; DEPTNO=40",
+        "EMPNO=120; NAME=Wilma; DEPTNO=20",
+        "EMPNO=130; NAME=Alice; DEPTNO=40");
+  }
+
+  /** Reads the DEPTS table. */
+  @Test public void testSalesDepts() throws SQLException {
+    final String sql = "select * from sales.depts";
+    checkSql("sales", sql,
+        "DEPTNO=10; NAME=Sales",
+        "DEPTNO=20; NAME=Marketing",
+        "DEPTNO=30; NAME=Accounts");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/test/resources/sales.json
----------------------------------------------------------------------
diff --git a/file/src/test/resources/sales.json b/file/src/test/resources/sales.json
index 9f8f2ec..7a93da1 100644
--- a/file/src/test/resources/sales.json
+++ b/file/src/test/resources/sales.json
@@ -1,3 +1,19 @@
+/*
+ * 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.
+ */
 {
   "version": "1.0",
   "defaultSchema": "SALES",
@@ -8,10 +24,10 @@
     "operand": {
       "tables": [ {
         "name": "EMPS",
-        "url": "file:file/src/test/resources/sales/EMPS.html"
+        "url": "file:sales/EMPS.html"
       }, {
         "name": "DEPTS",
-        "url": "file:file/src/test/resources/sales/DEPTS.html"
+        "url": "file:sales/DEPTS.html"
       } ]
     }
   } ]

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/test/resources/testModel.json
----------------------------------------------------------------------
diff --git a/file/src/test/resources/testModel.json b/file/src/test/resources/testModel.json
index f88b661..d7f286a 100644
--- a/file/src/test/resources/testModel.json
+++ b/file/src/test/resources/testModel.json
@@ -1,3 +1,19 @@
+/*
+ * 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.
+ */
 {
   "version": "1.0",
   "defaultSchema": "TEST",
@@ -8,13 +24,13 @@
     "operand": {
       "tables": [ {
         "name": "T1",
-        "url": "file:target/test-classes/tableOK.html"
+        "url": "file:tableOK.html"
       }, {
         "name": "T1_NO_TH",
-        "url": "file:target/test-classes/tableNoTH.html"
+        "url": "file:tableNoTH.html"
       }, {
         "name": "TABLEX2",
-        "url": "file:target/test-classes/tableX2.html"
+        "url": "file:tableX2.html"
       } ]
     }
   } ]

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/file/src/test/resources/wiki.json
----------------------------------------------------------------------
diff --git a/file/src/test/resources/wiki.json b/file/src/test/resources/wiki.json
index 36654e9..712b3c4 100644
--- a/file/src/test/resources/wiki.json
+++ b/file/src/test/resources/wiki.json
@@ -1,10 +1,26 @@
+/*
+ * 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.
+ */
 {
 /*
  * Example calcite-file schema using wikipedia pages for us states and
  * cities.
  */
-  version: "1.0",
-  defaultSchema: "WIKI",
+  "version": "1.0",
+  "defaultSchema": "WIKI",
   /*
    * Schema Definition
    *
@@ -78,14 +94,33 @@
         "selector": "#mw-content-text > table.wikitable.sortable",
         "index": 0,
         "fields": [
+          {"th": "State & abbreviation", "name": "State", "selector": "a", "selectedElement": 0},
+          {"th": "Statehood[A]", "name": "Statehood", "type": "Date"},
+          {"th": "Largest city[A][10]", "name": "Largest City"},
+          {"th": "Population (2013 est)[11]", "name": "Population", "type": "double"},
+          {"th": "Total area in mi2 (km2)[C][12]", "name": "Total Area", "type": "double"},
+          {"th": "Land area in mi2 (km2)[C][12]", "name": "Land Area", "type": "double"},
+          {"th": "Water area in mi2 (km2)[C][12]", "name": "Water Area",
+              "type": "double"}
+        ]
+      }, {
+        // Historic snapshot of "states" page, to ensure test stability.
+        "name": "States_as_of",
+        "url": "https://en.wikipedia.org/w/index.php?title=List_of_states_and_territories_of_the_United_States&oldid=713721917",
+        "selector": "#mw-content-text > table.wikitable.sortable",
+        "index": 0,
+        "fields": [
           {"th": "State", "selector": "a", "selectedElement": 0},
+          {"th": "Capital"},
+          {"th": "Abbr.", "name": "Abbreviation"},
           {"th": "Largest city[A][10]", "name": "Largest City"},
           {"th": "Statehood[B]", "name": "Statehood", "type": "Date"},
           {"th": "Population (2013 est)[11]", "name": "Population", "type": "double"},
           {"th": "Total area in mi2 (km2)[C][12]", "name": "Total Area", "type": "double"},
           {"th": "Land area in mi2 (km2)[C][12]", "name": "Land Area", "type": "double"},
           {"th": "Water area in mi2 (km2)[C][12]", "name": "Water Area",
-              "type": "double"}
+              "type": "double"},
+          {"th": "House seat(s)", "name": "House seats", "type": "double"}
         ]
       } ]
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/24c25fc3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 977b8ea..9297506 100644
--- a/pom.xml
+++ b/pom.xml
@@ -94,10 +94,14 @@ limitations under the License.
     <junit.version>4.12</junit.version>
     <maven-checkstyle-plugin.version>2.12.1</maven-checkstyle-plugin.version>
     <maven-dependency-plugin.version>2.10</maven-dependency-plugin.version>
+    <!-- Apache 18 has 3.0.1, but need 3.0.2 for [MSOURCES-94]. -->
+    <maven-jar-plugin.version>3.0.2</maven-jar-plugin.version>
     <!-- Apache 18 has 2.10.3, but need 2.10.4 for [MJAVADOC-442]. -->
     <maven-javadoc-plugin.version>2.10.4</maven-javadoc-plugin.version>
     <maven-scm-provider.version>1.9.4</maven-scm-provider.version>
     <maven-shade-plugin.version>2.1</maven-shade-plugin.version>
+    <!-- Apache 18 has 3.0.0, but need 3.0.1 for [MSOURCES-94]. -->
+    <maven-source-plugin.version>3.0.1</maven-source-plugin.version>
     <mockito-all.version>1.10.19</mockito-all.version>
     <mongo-java-driver.version>2.12.3</mongo-java-driver.version>
     <mysql-driver.version>5.1.20</mysql-driver.version>
@@ -699,7 +703,7 @@ limitations under the License.
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-jar-plugin</artifactId>
-          <version>3.0.2</version>
+          <version>${maven-jar-plugin.version}</version>
           <configuration>
             <archive>
               <manifest>
@@ -717,7 +721,7 @@ limitations under the License.
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-source-plugin</artifactId>
-          <version>3.0.1</version>
+          <version>${maven-source-plugin.version}</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>