Skip to content

Commit

Permalink
[SPARK-2706][SQL] Enable Spark to support Hive 0.13
Browse files Browse the repository at this point in the history
Given that a lot of users are trying to use hive 0.13 in spark, and the incompatibility between hive-0.12 and hive-0.13 on the API level I want to propose following approach, which has no or minimum impact on existing hive-0.12 support, but be able to jumpstart the development of hive-0.13 and future version support.

Approach: Introduce “hive-version” property,  and manipulate pom.xml files to support different hive version at compiling time through shim layer, e.g., hive-0.12.0 and hive-0.13.1. More specifically,

1. For each different hive version, there is a very light layer of shim code to handle API differences, sitting in sql/hive/hive-version, e.g., sql/hive/v0.12.0 or sql/hive/v0.13.1

2. Add a new profile hive-default active by default, which picks up all existing configuration and hive-0.12.0 shim (v0.12.0)  if no hive.version is specified.

3. If user specifies different version (currently only 0.13.1 by -Dhive.version = 0.13.1), hive-versions profile will be activated, which pick up hive-version specific shim layer and configuration, mainly the hive jars and hive-version shim, e.g., v0.13.1.

4. With this approach, nothing is changed with current hive-0.12 support.

No change by default: sbt/sbt -Phive
For example: sbt/sbt -Phive -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly

To enable hive-0.13: sbt/sbt -Dhive.version=0.13.1
For example: sbt/sbt -Dhive.version=0.13.1 -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly

Note that in hive-0.13, hive-thriftserver is not enabled, which should be fixed by other Jira, and we don’t need -Phive with -Dhive.version in building (probably we should use -Phive -Dhive.version=xxx instead after thrift server is also supported in hive-0.13.1).

Author: Zhan Zhang <zhazhan@gmail.com>
Author: zhzhan <zhazhan@gmail.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes apache#2241 from zhzhan/spark-2706 and squashes the following commits:

3ece905 [Zhan Zhang] minor fix
410b668 [Zhan Zhang] solve review comments
cbb4691 [Zhan Zhang] change run-test for new options
0d4d2ed [Zhan Zhang] rebase
497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
8fad1cf [Zhan Zhang] change the pom file and make hive-0.13.1 as the default
ab028d1 [Zhan Zhang] rebase
4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
4cb1b93 [zhzhan] Merge pull request #1 from pwendell/pr-2241
b0478c0 [Patrick Wendell] Changes to simplify the build of SPARK-2706
2b50502 [Zhan Zhang] rebase
a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb22863 [Zhan Zhang] correct the typo
20f6cf7 [Zhan Zhang] solve compatability issue
f7912a9 [Zhan Zhang] rebase and solve review feedback
301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
10c3565 [Zhan Zhang] address review comments
6bc9204 [Zhan Zhang] rebase and remove temparory repo
d3aa3f2 [Zhan Zhang] Merge branch 'master' into spark-2706
cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3ced0d7 [Zhan Zhang] rebase
d9b981d [Zhan Zhang] rebase and fix error due to rollback
adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3dd50e8 [Zhan Zhang] solve conflicts and remove unnecessary implicts
d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
dc7bdb3 [Zhan Zhang] solve conflicts
7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d7c3e1e [Zhan Zhang] Merge branch 'master' into spark-2706
68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d48bd18 [Zhan Zhang] address review comments
3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
57ea52e [Zhan Zhang] Merge branch 'master' into spark-2706
2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
9412d24 [Zhan Zhang] address review comments
f4af934 [Zhan Zhang] rebase
1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
128b60b [Zhan Zhang] ignore 0.12.0 test cases for the time being
af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
5f5619f [Zhan Zhang] restructure the directory and different hive version support
05d3683 [Zhan Zhang] solve conflicts
e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
94b4fdc [Zhan Zhang] Spark-2706: hive-0.13.1 support on spark
87ebf3b [Zhan Zhang] Merge branch 'master' into spark-2706
921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f896b2a [Zhan Zhang] Merge branch 'master' into spark-2706
789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f6a8a40 [Zhan Zhang] revert
ba14f28 [Zhan Zhang] test
dbedff3 [Zhan Zhang] Merge remote-tracking branch 'upstream/master'
70964fe [Zhan Zhang] revert
fe0f379 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark
70ffd93 [Zhan Zhang] revert
42585ec [Zhan Zhang] test
7d5fce2 [Zhan Zhang] test
  • Loading branch information
zhzhan authored and marmbrus committed Oct 24, 2014
1 parent 0e88661 commit 7c89a8f
Show file tree
Hide file tree
Showing 19 changed files with 406 additions and 63 deletions.
6 changes: 6 additions & 0 deletions assembly/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,12 @@
<artifactId>spark-hive_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<!-- TODO: Move this to "hive" profile once 0.13 JDBC is supported -->
<id>hive-0.12.0</id>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive-thriftserver_${scala.binary.version}</artifactId>
Expand Down
4 changes: 2 additions & 2 deletions dev/run-tests
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ CURRENT_BLOCK=$BLOCK_BUILD

{
# We always build with Hive because the PySpark Spark SQL tests need it.
BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive"
BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0"

echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS"

Expand All @@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS
# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled.
# This must be a single argument, as it is.
if [ -n "$_RUN_SQL_TESTS" ]; then
SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive"
SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0"
fi

if [ -n "$_SQL_TESTS_ONLY" ]; then
Expand Down
26 changes: 17 additions & 9 deletions docs/building-spark.md
Original file line number Diff line number Diff line change
Expand Up @@ -97,12 +97,20 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package
mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package
{% endhighlight %}

<!--- TODO: Update this when Hive 0.13 JDBC is added -->

# Building With Hive and JDBC Support
To enable Hive integration for Spark SQL along with its JDBC server and CLI,
add the `-Phive` profile to your existing build options.
add the `-Phive` profile to your existing build options. By default Spark
will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using
the `-Phive-0.12.0` profile. NOTE: currently the JDBC server is only
supported for Hive 0.12.0.
{% highlight bash %}
# Apache Hadoop 2.4.X with Hive support
# Apache Hadoop 2.4.X with Hive 13 support
mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package

# Apache Hadoop 2.4.X with Hive 12 support
mvn -Pyarn -Phive-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package
{% endhighlight %}

# Spark Tests in Maven
Expand All @@ -111,8 +119,8 @@ Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.o

Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence:

mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package
mvn -Pyarn -Phadoop-2.3 -Phive test
mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-0.12.0 clean package
mvn -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test

The ScalaTest plugin also supports running only a specific test suite as follows:

Expand Down Expand Up @@ -175,21 +183,21 @@ can be set to control the SBT build. For example:

Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence:

sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly
sbt/sbt -Pyarn -Phadoop-2.3 -Phive test
sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 assembly
sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test

To run only a specific test suite as follows:

sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite"
sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 "test-only org.apache.spark.repl.ReplSuite"

To run test suites of a specific sub project as follows:

sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test
sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 core/test

# Speeding up Compilation with Zinc

[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental
compiler. When run locally as a background process, it speeds up builds of Scala-based projects
like Spark. Developers who regularly recompile Spark with Maven will be the most interested in
Zinc. The project site gives instructions for building and running `zinc`; OS X users can
install it using `brew install zinc`.
install it using `brew install zinc`.
29 changes: 24 additions & 5 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,11 @@
<hbase.version>0.94.6</hbase.version>
<flume.version>1.4.0</flume.version>
<zookeeper.version>3.4.5</zookeeper.version>
<hive.version>0.12.0-protobuf-2.5</hive.version>
<!-- Version used in Maven Hive dependency -->
<hive.version>0.13.1</hive.version>
<!-- Version used for internal directory structure -->
<hive.version.short>0.13.1</hive.version.short>
<derby.version>10.10.1.1</derby.version>
<parquet.version>1.4.3</parquet.version>
<jblas.version>1.2.3</jblas.version>
<jetty.version>8.1.14.v20131031</jetty.version>
Expand Down Expand Up @@ -456,7 +460,7 @@
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
<version>10.4.2.0</version>
<version>${derby.version}</version>
</dependency>
<dependency>
<groupId>com.codahale.metrics</groupId>
Expand Down Expand Up @@ -1308,16 +1312,31 @@
</dependency>
</dependencies>
</profile>

<profile>
<id>hive</id>
<id>hive-0.12.0</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<!-- TODO: Move this to "hive" profile once 0.13 JDBC is supported -->
<modules>
<module>sql/hive-thriftserver</module>
</modules>
<properties>
<hive.version>0.12.0-protobuf-2.5</hive.version>
<hive.version.short>0.12.0</hive.version.short>
<derby.version>10.4.2.0</derby.version>
</properties>
</profile>
<profile>
<id>hive-0.13.1</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<properties>
<hive.version>0.13.1</hive.version>
<hive.version.short>0.13.1</hive.version.short>
<derby.version>10.10.1.1</derby.version>
</properties>
</profile>

</profiles>
</project>
37 changes: 31 additions & 6 deletions sql/hive/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,6 @@
</properties>

<dependencies>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-hive-bundle</artifactId>
<version>1.5.0</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
Expand Down Expand Up @@ -116,7 +111,6 @@
<scope>test</scope>
</dependency>
</dependencies>

<profiles>
<profile>
<id>hive</id>
Expand Down Expand Up @@ -144,6 +138,19 @@
</plugins>
</build>
</profile>
<profile>
<id>hive-0.12.0</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<dependencies>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-hive-bundle</artifactId>
<version>1.5.0</version>
</dependency>
</dependencies>
</profile>
</profiles>

<build>
Expand All @@ -154,6 +161,24 @@
<groupId>org.scalatest</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-default-sources</id>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>v${hive.version.short}/src/main/scala</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>

<!-- Deploy datanucleus jars to the spark/lib_managed/jars directory -->
<plugin>
Expand Down
23 changes: 10 additions & 13 deletions sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.metadata.Table
import org.apache.hadoop.hive.ql.processors._
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.ql.stats.StatsSetupConst
import org.apache.hadoop.hive.serde2.io.TimestampWritable
import org.apache.hadoop.hive.serde2.io.DateWritable

Expand All @@ -47,6 +46,7 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.{Command => PhysicalCommand}
import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
import org.apache.spark.sql.hive.HiveShim

/**
* DEPRECATED: Use HiveContext instead.
Expand Down Expand Up @@ -171,13 +171,15 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {

val tableParameters = relation.hiveQlTable.getParameters
val oldTotalSize =
Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)).map(_.toLong).getOrElse(0L)
Option(tableParameters.get(HiveShim.getStatsSetupConstTotalSize))
.map(_.toLong)
.getOrElse(0L)
val newTotalSize = getFileSizeForTable(hiveconf, relation.hiveQlTable)
// Update the Hive metastore if the total size of the table is different than the size
// recorded in the Hive metastore.
// This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats().
if (newTotalSize > 0 && newTotalSize != oldTotalSize) {
tableParameters.put(StatsSetupConst.TOTAL_SIZE, newTotalSize.toString)
tableParameters.put(HiveShim.getStatsSetupConstTotalSize, newTotalSize.toString)
val hiveTTable = relation.hiveQlTable.getTTable
hiveTTable.setParameters(tableParameters)
val tableFullName =
Expand Down Expand Up @@ -282,29 +284,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
*/
protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = {
try {
// Session state must be initilized before the CommandProcessor is created .
SessionState.start(sessionState)

val cmd_trimmed: String = cmd.trim()
val tokens: Array[String] = cmd_trimmed.split("\\s+")
val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim()
val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf)
val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf)

proc match {
case driver: Driver =>
driver.init()

val results = new JArrayList[String]
val results = HiveShim.createDriverResultsArray
val response: CommandProcessorResponse = driver.run(cmd)
// Throw an exception if there is an error in query processing.
if (response.getResponseCode != 0) {
driver.destroy()
driver.close()
throw new QueryExecutionException(response.getErrorMessage)
}
driver.setMaxRows(maxRows)
driver.getResults(results)
driver.destroy()
results
driver.close()
HiveShim.processResults(results)
case _ =>
sessionState.out.println(tokens(0) + " " + cmd_1)
Seq(proc.run(cmd_1).getResponseCode.toString)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.hadoop.{io => hadoopIo}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.hive.HiveShim

/* Implicit conversions */
import scala.collection.JavaConversions._
Expand Down Expand Up @@ -149,7 +150,7 @@ private[hive] trait HiveInspectors {
case l: Long => l: java.lang.Long
case l: Short => l: java.lang.Short
case l: Byte => l: java.lang.Byte
case b: BigDecimal => new HiveDecimal(b.underlying())
case b: BigDecimal => HiveShim.createDecimal(b.underlying())
case b: Array[Byte] => b
case d: java.sql.Date => d
case t: java.sql.Timestamp => t
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import scala.util.parsing.combinator.RegexParsers
import org.apache.hadoop.hive.metastore.api.{FieldSchema, SerDeInfo, StorageDescriptor, Partition => TPartition, Table => TTable}
import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.hadoop.hive.ql.stats.StatsSetupConst
import org.apache.hadoop.hive.serde2.Deserializer

import org.apache.spark.Logging
Expand All @@ -34,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.util.Utils

/* Implicit conversions */
Expand All @@ -56,7 +56,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
val table = client.getTable(databaseName, tblName)
val partitions: Seq[Partition] =
if (table.isPartitioned) {
client.getAllPartitionsForPruner(table).toSeq
HiveShim.getAllPartitionsOf(client, table).toSeq
} else {
Nil
}
Expand Down Expand Up @@ -185,7 +185,7 @@ object HiveMetastoreTypes extends RegexParsers {
"bigint" ^^^ LongType |
"binary" ^^^ BinaryType |
"boolean" ^^^ BooleanType |
"decimal" ^^^ DecimalType |
HiveShim.metastoreDecimal ^^^ DecimalType |
"date" ^^^ DateType |
"timestamp" ^^^ TimestampType |
"varchar\\((\\d+)\\)".r ^^^ StringType
Expand Down Expand Up @@ -272,13 +272,13 @@ private[hive] case class MetastoreRelation
// of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`,
// `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future.
BigInt(
Option(hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE))
Option(hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize))
.map(_.toLong)
.getOrElse(sqlContext.defaultSizeInBytes))
}
)

val tableDesc = new TableDesc(
val tableDesc = HiveShim.getTableDesc(
Class.forName(
hiveQlTable.getSerializationLib,
true,
Expand Down
16 changes: 14 additions & 2 deletions sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,8 @@
package org.apache.spark.sql.hive

import java.sql.Date

import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.Context
import org.apache.hadoop.hive.ql.lib.Node
import org.apache.hadoop.hive.ql.parse._
import org.apache.hadoop.hive.ql.plan.PlanUtils
Expand Down Expand Up @@ -216,7 +217,18 @@ private[hive] object HiveQl {
/**
* Returns the AST for the given SQL string.
*/
def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql))
def getAst(sql: String): ASTNode = {
/*
* Context has to be passed in hive0.13.1.
* Otherwise, there will be Null pointer exception,
* when retrieving properties form HiveConf.
*/
val hContext = new Context(new HiveConf())
val node = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, hContext))
hContext.clear()
node
}


/** Returns a LogicalPlan for a given HiveQL string. */
def parseSql(sql: String): LogicalPlan = hqlParser(sql)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import org.apache.spark.SerializableWritable
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.hive.HiveShim

/**
* A trait for subclasses that handle table scans.
Expand Down Expand Up @@ -138,7 +139,7 @@ class HadoopTableReader(
filterOpt: Option[PathFilter]): RDD[Row] = {
val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) =>
val partDesc = Utilities.getPartitionDesc(partition)
val partPath = partition.getPartitionPath
val partPath = HiveShim.getDataLocationPath(partition)
val inputPathStr = applyFilterIfNeeded(partPath, filterOpt)
val ifc = partDesc.getInputFileFormatClass
.asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]]
Expand Down
Loading

0 comments on commit 7c89a8f

Please sign in to comment.