Databricks Runtime 4.1 (unsupported)

Databricks released this image in May 2018.

Important

This release was deprecated on January 17, 2019. For more information about the Databricks Runtime deprecation policy and schedule, see Databricks runtime support lifecycles.

The following release notes provide information about Databricks Runtime 4.1, powered by Apache Spark.

Delta Lake

Databricks Runtime version 4.1 adds major quality improvements and functionality to Delta Lake. Databricks highly recommends that all Delta Lake customers upgrade to the new runtime. This release remains in Private Preview, but it represents a candidate release in anticipation of the upcoming general availability (GA) release.

Breaking changes

  • Databricks Runtime 4.1 includes changes to the transaction protocol to enable new features, such as validation. Tables created with Databricks Runtime 4.1 automatically use the new version and cannot be written to by older versions of Databricks Runtime. You must upgrade existing tables in order to take advantage of these improvements. To upgrade an existing table, first upgrade all jobs that are writing to the table. Then run:

    com.databricks.delta.Delta.upgradeTableProtocol("<path-to-table>" or "<tableName>")
    

    See How does Databricks manage Delta Lake feature compatibility? for more information.

  • Writes are now validated against the current schema of the table rather than, as before, automatically adding columns that are missing from the destination table. To enable the previous behavior, set the mergeSchema option to true. See Delta Lake schema validation.

  • If you are running earlier versions of Databricks Delta, you must upgrade all jobs before you use Databricks Runtime 4.1. If you see either of these errors, upgrade to Databricks Runtime 4.1:

    java.lang.NumberFormatException: For input string: "00000000000000....crc"
    scala.MatchError
    
  • Tables can no longer have columns that differ only by case.

  • Delta-specific table configurations must now be prefixed with delta.

New features

  • Schema management - Databricks Delta now validates appends and overwrites to an existing table to ensure that the schema being written matches the schema.

    • Databricks Delta continues to support automatic schema migration.

    • Databricks Delta now supports the following DDL to modify schema explicitly:

      • ALTER TABLE ADD COLUMN to add new columns to a table

      • ALTER TABLE CHANGE COLUMNS to change column ordering

      • ALTER TABLE [SET|UNSET] TBLPROPERTIES

    For details, see Delta Lake schema validation.

  • Enhanced DDL and table support

    • Full support for table DDL and saveAsTable(). save() and saveAsTable() now have identical semantics.

    • All DDL and DML commands support both table name and delta.`<path-to-table>` .

    • SHOW PARTITIONS

    • SHOW COLUMNS

    • DESC TABLE

    • Detailed table information - You can see the current reader and writer versions of a table by running DESCRIBE DETAIL. See How does Databricks manage Delta Lake feature compatibility?.

    • Table details - Provenance information is now available for each write to a table. The Data sidebar also shows detailed table information and history for Databricks Delta tables. See Review Delta Lake table details with describe detail.

    • Streaming tables - Streaming DataFrames can be created using spark.readStream.format("delta").table("<table-name>").

    • Append-only tables - Databricks Delta now supports basic data governance. You can block deletes and modifications to a table by setting the table property delta.appendOnly=true.

    • MERGE INTO source - Adds more comprehensive support to the source query specification of MERGE. For example, you can specify LIMIT, ORDER BY and INLINE TABLE in the source.

    • Full support for Table ACLs.

Performance improvements

  • Reduced stats collection overhead - The efficiency of stats collection has been improved and stats are now only collected for a configurable number of columns, set to 32 by default. Databricks Delta write performance has been improved by up to 2x due to the reduction in stats collection overhead. To configure the number of columns, set the table property delta.dataSkippingNumIndexedCols=<number-of-columns>.

  • Support for limit push down - Statistics are used to limit the number of files scanned for queries that have LIMIT and predicates over partition columns. This is applicable to queries in notebooks due to the implicit limit=1000 in effect for all notebook commands.

  • Filter pushdown in the streaming source - Streaming queries now use partitioning when starting a new stream to skip irrelevant data.

  • Improved parallelism for OPTIMIZE - OPTIMIZE now runs as a single Spark task and will use all parallelism available on the cluster (previously was limited to 100 compacted files at a time).

  • Data Skipping in DML - UPDATE, DELETE, and MERGE now use statistics when locating files that need to be rewritten.

  • Randomized S3 prefixes - To avoid hot spots in S3 metadata for large tables, you can now set the table property delta.randomizeFilePrefixes=true.

  • Decreased checkpoint retention - checkpoints are now retained for two days (history is still retained for 30) to decrease storage costs for the transaction log.

API behavior

  • The behavior of insertInto(<table-name>) in Databricks Delta is the same as other data sources.

    • If no mode is specified or mode is ErrorIfExists, Ignore, or Append, appends the data in the DataFrame to the Databricks Delta table.

    • If mode is Overwrite, deletes all data in the existing table and inserts the data from the DataFrame into the Databricks Delta table.

  • If cached, the destination table of MERGE must be manually uncached.

Usability improvements

  • Workload migration validations - Common mistakes made when migrating workloads to Databricks Delta now throw an exception rather than failing:

    • Using format("parquet") to read or write a table.

    • Reading or writing directly to a partition (that is, /path/to/delta/part=1).

    • Vacuuming subdirectories of a table.

    • INSERT OVERWRITE DIRECTORY using Parquet on a table.

  • Case insensitive configuration - Options to the DataFrame Reader/Writer and table properties are now case insensitive (including both read path and write path).

  • Column names - Table column names can now include dots.

Known issues

Warning

Writes to a single table must originate from a single cluster. There is experimental support for writes from multiple clusters in the same workspace. Contact Databricks support if you are interested in trying this feature.

  • The inserts of multi-insert statements are in different units of work, rather than the same transaction.

Bug fixes

  • An infinite loop when starting a new stream on a fast updating table has been fixed.

Deprecations

Structured Streaming does not handle input that is not an append and throws an exception if any modifications occur on the table being used as a source. Previously you could override this behavior using the ignoreFileDeletion flag, but it is now deprecated. Instead use ignoreDeletes or ignoreChanges. See Delta table as a source.

Other Changes and Improvements

  • Query Watchdog is enabled for all all-purpose clusters created using the UI.

  • Improved driver-side performance for the DBIO cache

  • Improved performance for Parquet decoding via a new native Parquet decoder

  • Improved performance for common subexpression elimination

  • Improved data skipping performance for large table joining small tables (fact-dimension table joins)

  • display() now renders columns containing image data types as rich HTML.

  • New s3select data source for leveraging S3 Select

  • Improvements to Log, load, register, and deploy MLflow models

    • Upgraded dbml-local to latest release 0.4.1

    • Fixed bug with models exported with threshold parameter specified

    • Added support for exporting OneVsRestModel, GBTClassificationModel

  • Upgraded some installed Python libraries:

    • pip: from 9.0.1 to 10.0.0b2

    • setuptools: from 38.5.1 to 39.0.1

    • tornado: 4.5.3 to 5.0.1

    • wheel: 0.30.0 to 0.31.0

  • Upgraded several installed R libraries. See Installed R Libraries.

  • Upgraded AWS Java SDK from 1.11.253 to 1.11.313.

  • Upgraded Azure Data Lake Store SDK from 2.0.11 to 2.2.8.

  • Upgraded CUDA to 9.0 from 8.0 and CUDNN to 7.0 from 6.0 for GPU clusters.

  • For GPU clusters, GPU metrics now appear in the Cluster metrics Ganglia UI.

Apache Spark

Databricks Runtime 4.1 includes Apache Spark 2.3.0. This release includes all fixes and improvements included in Databricks Runtime 4.0 (unsupported), as well as the following additional bug fixes and improvements made to Spark:

  • [SPARK-24007][SQL] EqualNullSafe for FloatType and DoubleType might generate a wrong result by codegen.

  • [SPARK-23942][PYTHON][SQL] Makes collect in PySpark as action for a query executor listener

  • [SPARK-23815][CORE] Spark writer dynamic partition overwrite mode may fail to write output on multi level partition

  • [SPARK-23748][SS] Fix SS continuous process doesn’t support SubqueryAlias issue

  • [SPARK-23963][SQL] Properly handle large number of columns in query on text-based Hive table

  • [SPARK-23867][SCHEDULER] use droppedCount in logWarning

  • [SPARK-23816][CORE] Killed tasks should ignore FetchFailures.

  • [SPARK-23809][SQL] Active SparkSession should be set by getOrCreate

  • [SPARK-23966][SS] Refactoring all checkpoint file writing logic in a common CheckpointFileManager interface

  • [SPARK-21351][SQL] Update nullability based on children’s output

  • [SPARK-23847][PYTHON][SQL] Add asc_nulls_first, asc_nulls_last to PySpark

  • [SPARK-23822][SQL] Improve error message for Parquet schema mismatches

  • [SPARK-23823][SQL] Keep origin in transformExpression

  • [SPARK-23838][WEBUI] Running SQL query is displayed as “completed” in SQL tab

  • [SPARK-23802][SQL] PropagateEmptyRelation can leave query plan in unresolved state

  • [SPARK-23727][SQL] Support for pushing down filters for DateType in parquet

  • [SPARK-23574][SQL] Report SinglePartition in DataSourceV2ScanExec when there’s exactly 1 data reader factory.

  • [SPARK-23533][SS] Add support for changing ContinuousDataReader’s startOffset

  • [SPARK-23491][SS] Remove explicit job cancellation from ContinuousExecution reconfiguring

  • [SPARK-23040][CORE] Returns interruptible iterator for shuffle reader

  • [SPARK-23827][SS] StreamingJoinExec should ensure that input data is partitioned into specific number of partitions

  • [SPARK-23639][SQL] Obtain token before init metastore client in SparkSQL CLI

  • [SPARK-23806]Broadcast.unpersist can cause fatal exception when used…

  • [SPARK-23599][SQL] Use RandomUUIDGenerator in Uuid expression

  • [SPARK-23599][SQL] Add a UUID generator from Pseudo-Random Numbers

  • [SPARK-23759][UI] Unable to bind Spark UI to specific host name / IP

  • [SPARK-23769][CORE] Remove comments that unnecessarily disable Scalastyle check

  • [SPARK-23614][SQL] Fix incorrect reuse exchange when caching is used

  • [SPARK-23760][SQL] CodegenContext.withSubExprEliminationExprs should save/restore CSE state correctly

  • [SPARK-23729][CORE] Respect URI fragment when resolving globs

  • [SPARK-23550][CORE] Cleanup Utils

  • [SPARK-23288][SS] Fix output metrics with parquet sink

  • [SPARK-23264][SQL] Fix scala.MatchError in literals.sql.out

  • [SPARK-23649][SQL] Skipping chars disallowed in UTF-8

  • [SPARK-23691][PYTHON] Use sql_conf util in PySpark tests where possible

  • [SPARK-23644][CORE][UI] Use absolute path for REST call in SHS

  • [SPARK-23706][PYTHON] spark.conf.get(value, default=None) should produce None in PySpark

  • [SPARK-23623][SS] Avoid concurrent use of cached consumers in CachedKafkaConsumer

  • [SPARK-23670][SQL] Fix memory leak on SparkPlanGraphWrapper

  • [SPARK-23608][CORE][WEBUI] Add synchronization in SHS between attachSparkUI and detachSparkUI functions to avoid concurrent modification issue to Jetty Handlers

  • [SPARK-23671][CORE] Fix condition to enable the SHS thread pool.

  • [SPARK-23658][LAUNCHER] InProcessAppHandle uses the wrong class in getLogger

  • [SPARK-23642][DOCS] AccumulatorV2 subclass isZero scaladoc fix

  • [SPARK-22915][MLLIB] Streaming tests for spark.ml.feature, from N to Z

  • [SPARK-23598][SQL] Make methods in BufferedRowIterator public to avoid runtime error for a large query

  • [SPARK-23546][SQL] Refactor stateless methods/values in CodegenContext

  • [SPARK-23523][SQL] Fix the incorrect result caused by the rule OptimizeMetadataOnlyQuery

  • [SPARK-23462][SQL] improve missing field error message in StructType

  • [SPARK-23624][SQL] Revise doc of method pushFilters in Datasource V2

  • [SPARK-23173][SQL] Avoid creating corrupt parquet files when loading data from JSON

  • [SPARK-23436][SQL] Infer partition as Date only if it can be casted to Date

  • [SPARK-23406][SS] Enable stream-stream self-joins

  • [SPARK-23490][SQL] Check storage.locationUri with existing table in CreateTable

  • [SPARK-23524]Big local shuffle blocks should not be checked for corruption.

  • [SPARK-23525][SQL] Support ALTER TABLE CHANGE COLUMN COMMENT for external hive table

  • [SPARK-23434][SQL] Spark should not warn metadata directory for a HDFS file path

  • [SPARK-23457][SQL] Register task completion listeners first in ParquetFileFormat

  • [SPARK-23329][SQL] Fix documentation of trigonometric functions

  • [SPARK-23569][PYTHON] Allow pandas_udf to work with python3 style type-annotated functions

  • [SPARK-23570][SQL] Add Spark 2.3.0 in HiveExternalCatalogVersionsSuite

  • [SPARK-23517][PYTHON] Make pyspark.util.exception_message_ produce the trace from Java side by Py4JJavaError

  • [SPARK-23508][CORE] Fix BlockmanagerId in case blockManagerIdCache cause oom

  • [SPARK-23448][SQL] Clarify JSON and CSV parser behavior in document

  • [SPARK-23365][CORE] Do not adjust num executors when killing idle executors.

  • [SPARK-23438][DSTREAMS] Fix DStreams data loss with WAL when driver crashes

  • [SPARK-23475][UI] Show also skipped stages

  • [SPARK-23518][SQL] Avoid metastore access when the users only want to read and write data frames

  • [SPARK-23406][SS] Enable stream-stream self-joins

  • [SPARK-23541][SS] Allow Kafka source to read data with greater parallelism than the number of article-partitions

  • [SPARK-23097][SQL][SS] Migrate text socket source to V2

  • [SPARK-23362][SS] Migrate Kafka Microbatch source to v2

  • [SPARK-23445]ColumnStat refactoring

  • [SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs

  • [SPARK-23447][SQL] Cleanup codegen template for Literal

  • [SPARK-23366]Improve hot reading path in ReadAheadInputStream

  • [SPARK-22624][PYSPARK] Expose range partitioning shuffle

System Environment

  • Operating System: Ubuntu 16.04.4 LTS

  • Java: 1.8.0_162

  • Scala: 2.11.8

  • Python: 2.7.12 for Python 2 clusters and 3.5.2 for Python 3 clusters.

  • R: R version 3.4.4 (2018-03-15)

  • GPU clusters: The following NVIDIA GPU libraries are installed:

    • Tesla driver 375.66

    • CUDA 9.0

    • cuDNN 7.0

Installed Python Libraries

Library

Version

Library

Version

Library

Version

ansi2html

1.1.1

argparse

1.2.1

backports-abc

0.5

boto

2.42.0

boto3

1.4.1

botocore

1.4.70

brewer2mpl

1.4.1

certifi

2016.2.28

cffi

1.7.0

chardet

2.3.0

colorama

0.3.7

configobj

5.0.6

cryptography

1.5

cycler

0.10.0

Cython

0.24.1

decorator

4.0.10

docutils

0.14

enum34

1.1.6

et-xmlfile

1.0.1

freetype-py

1.0.2

funcsigs

1.0.2

fusepy

2.0.4

futures

3.2.0

ggplot

0.6.8

html5lib

0.999

idna

2.1

ipaddress

1.0.16

ipython

2.2.0

ipython-genutils

0.1.0

jdcal

1.2

Jinja2

2.8

jmespath

0.9.0

llvmlite

0.13.0

lxml

3.6.4

MarkupSafe

0.23

matplotlib

1.5.3

mpld3

0.2

msgpack-python

0.4.7

ndg-httpsclient

0.3.3

numba

0.28.1

numpy

1.11.1

openpyxl

2.3.2

pandas

0.19.2

pathlib2

2.1.0

patsy

0.4.1

pexpect

4.0.1

pickleshare

0.7.4

Pillow

3.3.1

pip

10.0.0b2

ply

3.9

prompt-toolkit

1.0.7

psycopg2

2.6.2

ptyprocess

0.5.1

py4j

0.10.3

pyarrow

0.8.0

pyasn1

0.1.9

pycparser

2.14

Pygments

2.1.3

PyGObject

3.20.0

pyOpenSSL

16.0.0

pyparsing

2.2.0

pypng

0.0.18

Python

2.7.12

python-dateutil

2.5.3

python-geohash

0.8.5

pytz

2016.6.1

requests

2.11.1

s3transfer

0.1.9

scikit-learn

0.18.1

scipy

0.18.1

scour

0.32

seaborn

0.7.1

setuptools

39.0.1

simplejson

3.8.2

simples3

1.0

singledispatch

3.4.0.3

six

1.10.0

statsmodels

0.6.1

tornado

5.0.1

traitlets

4.3.0

urllib3

1.19.1

virtualenv

15.0.1

wcwidth

0.1.7

wheel

0.31.0

wsgiref

0.1.2

Installed R Libraries

Library

Version

Library

Version

Library

Version

abind

1.4-5

assertthat

0.2.0

backports

1.1.2

base

3.4.4

BH

1.66.0-1

bindr

0.1.1

bindrcpp

0.2.2

bit

1.1-12

bit64

0.9-7

bitops

1.0-6

blob

1.1.1

boot

1.3-20

brew

1.0-6

broom

0.4.4

car

3.0-0

carData

3.0-1

caret

6.0-79

cellranger

1.1.0

chron

2.3-52

class

7.3-14

cli

1.0.0

cluster

2.0.7

codetools

0.2-15

colorspace

1.3-2

commonmark

1.4

compiler

3.4.4

crayon

1.3.4

curl

3.2

CVST

0.2-1

data.table

1.10.4-3

datasets

3.4.4

DBI

0.8

ddalpha

1.3.1.1

DEoptimR

1.0-8

desc

1.1.1

devtools

1.13.5

dichromat

2.0-0

digest

0.6.15

dimRed

0.1.0

doMC

1.3.5

dplyr

0.7.4

DRR

0.0.3

forcats

0.3.0

foreach

1.4.4

foreign

0.8-69

gbm

2.1.3

ggplot2

2.2.1

git2r

0.21.0

glmnet

2.0-16

glue

1.2.0

gower

0.1.2

graphics

3.4.4

grDevices

3.4.4

grid

3.4.4

gsubfn

0.7

gtable

0.2.0

h2o

3.16.0.2

haven

1.1.1

hms

0.4.2

httr

1.3.1

hwriter

1.3.2

hwriterPlus

1.0-3

ipred

0.9-6

iterators

1.0.9

jsonlite

1.5

kernlab

0.9-25

KernSmooth

2.23-15

labeling

0.3

lattice

0.20-35

lava

1.6.1

lazyeval

0.2.1

littler

0.3.3

lme4

1.1-17

lubridate

1.7.3

magrittr

1.5

mapproj

1.2.6

maps

3.3.0

maptools

0.9-2

MASS

7.3-49

Matrix

1.2-13

MatrixModels

0.4-1

memoise

1.1.0

methods

3.4.4

mgcv

1.8-23

mime

0.5

minqa

1.2.4

mnormt

1.5-5

ModelMetrics

1.1.0

munsell

0.4.3

mvtnorm

1.0-7

nlme

3.1-137

nloptr

1.0.4

nnet

7.3-12

numDeriv

2016.8-1

openssl

1.0.1

openxlsx

4.0.17

parallel

3.4.4

pbkrtest

0.4-7

pillar

1.2.1

pkgconfig

2.0.1

pkgKitten

0.1.4

plogr

0.2.0

plyr

1.8.4

praise

1.0.0

prettyunits

1.0.2

pROC

1.11.0

prodlim

1.6.1

proto

1.0.0

psych

1.8.3.3

purrr

0.2.4

quantreg

5.35

R.methodsS3

1.7.1

R.oo

1.21.0

R.utils

2.6.0

R6

2.2.2

randomForest

4.6-14

RColorBrewer

1.1-2

Rcpp

0.12.16

RcppEigen

0.3.3.4.0

RcppRoll

0.2.2

RCurl

1.95-4.10

readr

1.1.1

readxl

1.0.0

recipes

0.1.2

rematch

1.0.1

reshape2

1.4.3

rio

0.5.10

rlang

0.2.0

robustbase

0.92-8

RODBC

1.3-15

roxygen2

6.0.1

rpart

4.1-13

rprojroot

1.3-2

Rserve

1.7-3

RSQLite

2.1.0

rstudioapi

0.7

scales

0.5.0

sfsmisc

1.1-2

sp

1.2-7

SparkR

2.3.0

SparseM

1.77

spatial

7.3-11

splines

3.4.4

sqldf

0.4-11

SQUAREM

2017.10-1

statmod

1.4.30

stats

3.4.4

stats4

3.4.4

stringi

1.1.7

stringr

1.3.0

survival

2.41-3

tcltk

3.4.4

TeachingDemos

2.10

testthat

2.0.0

tibble

1.4.2

tidyr

0.8.0

tidyselect

0.2.4

timeDate

3043.102

tools

3.4.4

utf8

1.1.3

utils

3.4.4

viridisLite

0.3.0

whisker

0.3-2

withr

2.1.2

xml2

1.2.0

Installed Java and Scala libraries (Scala 2.11 cluster version)

Group ID

Artifact ID

Version

antlr

antlr

2.7.7

com.amazonaws

amazon-kinesis-client

1.7.3

com.amazonaws

aws-java-sdk-autoscaling

1.11.313

com.amazonaws

aws-java-sdk-cloudformation

1.11.313

com.amazonaws

aws-java-sdk-cloudfront

1.11.313

com.amazonaws

aws-java-sdk-cloudhsm

1.11.313

com.amazonaws

aws-java-sdk-cloudsearch

1.11.313

com.amazonaws

aws-java-sdk-cloudtrail

1.11.313

com.amazonaws

aws-java-sdk-cloudwatch

1.11.313

com.amazonaws

aws-java-sdk-cloudwatchmetrics

1.11.313

com.amazonaws

aws-java-sdk-codedeploy

1.11.313

com.amazonaws

aws-java-sdk-cognitoidentity

1.11.313

com.amazonaws

aws-java-sdk-cognitosync

1.11.313

com.amazonaws

aws-java-sdk-config

1.11.313

com.amazonaws

aws-java-sdk-core

1.11.313

com.amazonaws

aws-java-sdk-datapipeline

1.11.313

com.amazonaws

aws-java-sdk-directconnect

1.11.313

com.amazonaws

aws-java-sdk-directory

1.11.313

com.amazonaws

aws-java-sdk-dynamodb

1.11.313

com.amazonaws

aws-java-sdk-ec2

1.11.313

com.amazonaws

aws-java-sdk-ecs

1.11.313

com.amazonaws

aws-java-sdk-efs

1.11.313

com.amazonaws

aws-java-sdk-elasticache

1.11.313

com.amazonaws

aws-java-sdk-elasticbeanstalk

1.11.313

com.amazonaws

aws-java-sdk-elasticloadbalancing

1.11.313

com.amazonaws

aws-java-sdk-elastictranscoder

1.11.313

com.amazonaws

aws-java-sdk-emr

1.11.313

com.amazonaws

aws-java-sdk-glacier

1.11.313

com.amazonaws

aws-java-sdk-iam

1.11.313

com.amazonaws

aws-java-sdk-importexport

1.11.313

com.amazonaws

aws-java-sdk-kinesis

1.11.313

com.amazonaws

aws-java-sdk-kms

1.11.313

com.amazonaws

aws-java-sdk-lambda

1.11.313

com.amazonaws

aws-java-sdk-logs

1.11.313

com.amazonaws

aws-java-sdk-machinelearning

1.11.313

com.amazonaws

aws-java-sdk-opsworks

1.11.313

com.amazonaws

aws-java-sdk-rds

1.11.313

com.amazonaws

aws-java-sdk-redshift

1.11.313

com.amazonaws

aws-java-sdk-route53

1.11.313

com.amazonaws

aws-java-sdk-s3

1.11.313

com.amazonaws

aws-java-sdk-ses

1.11.313

com.amazonaws

aws-java-sdk-simpledb

1.11.313

com.amazonaws

aws-java-sdk-simpleworkflow

1.11.313

com.amazonaws

aws-java-sdk-sns

1.11.313

com.amazonaws

aws-java-sdk-sqs

1.11.313

com.amazonaws

aws-java-sdk-ssm

1.11.313

com.amazonaws

aws-java-sdk-storagegateway

1.11.313

com.amazonaws

aws-java-sdk-sts

1.11.313

com.amazonaws

aws-java-sdk-support

1.11.313

com.amazonaws

aws-java-sdk-swf-libraries

1.11.22

com.amazonaws

aws-java-sdk-workspaces

1.11.313

com.amazonaws

jmespath-java

1.11.313

com.carrotsearch

hppc

0.7.2

com.chuusai

shapeless_2.11

2.3.2

com.clearspring.analytics

stream

2.7.0

com.databricks

Rserve

1.8-3

com.databricks

dbml-local_2.11

0.4.1-db1-spark2.3

com.databricks

dbml-local_2.11-tests

0.4.1-db1-spark2.3

com.databricks

jets3t

0.7.1-0

com.databricks.scalapb

compilerplugin_2.11

0.4.15-9

com.databricks.scalapb

scalapb-runtime_2.11

0.4.15-9

com.esotericsoftware

kryo-shaded

3.0.3

com.esotericsoftware

minlog

1.3.0

com.fasterxml

classmate

1.0.0

com.fasterxml.jackson.core

jackson-annotations

2.6.7

com.fasterxml.jackson.core

jackson-core

2.6.7

com.fasterxml.jackson.core

jackson-databind

2.6.7.1

com.fasterxml.jackson.dataformat

jackson-dataformat-cbor

2.6.7

com.fasterxml.jackson.datatype

jackson-datatype-joda

2.6.7

com.fasterxml.jackson.module

jackson-module-paranamer

2.6.7

com.fasterxml.jackson.module

jackson-module-scala_2.11

2.6.7.1

com.github.fommil

jniloader

1.1

com.github.fommil.netlib

core

1.1.2

com.github.fommil.netlib

native_ref-java

1.1

com.github.fommil.netlib

native_ref-java-natives

1.1

com.github.fommil.netlib

native_system-java

1.1

com.github.fommil.netlib

native_system-java-natives

1.1

com.github.fommil.netlib

netlib-native_ref-linux-x86_64-natives

1.1

com.github.fommil.netlib

netlib-native_system-linux-x86_64-natives

1.1

com.github.luben

zstd-jni

1.3.2-2

com.github.rwl

jtransforms

2.4.0

com.google.code.findbugs

jsr305

2.0.1

com.google.code.gson

gson

2.2.4

com.google.guava

guava

15.0

com.google.protobuf

protobuf-java

2.6.1

com.googlecode.javaewah

JavaEWAH

0.3.2

com.h2database

h2

1.3.174

com.jamesmurty.utils

java-xmlbuilder

1.1

com.jcraft

jsch

0.1.50

com.jolbox

bonecp

0.8.0.RELEASE

com.mchange

c3p0

0.9.5.1

com.mchange

mchange-commons-java

0.2.10

com.microsoft.azure

azure-data-lake-store-sdk

2.2.8

com.microsoft.sqlserver

mssql-jdbc

6.2.2.jre8

com.ning

compress-lzf

1.0.3

com.sun.mail

javax.mail

1.5.2

com.thoughtworks.paranamer

paranamer

2.8

com.trueaccord.lenses

lenses_2.11

0.3

com.twitter

chill-java

0.8.4

com.twitter

chill_2.11

0.8.4

com.twitter

parquet-hadoop-bundle

1.6.0

com.twitter

util-app_2.11

6.23.0

com.twitter

util-core_2.11

6.23.0

com.twitter

util-jvm_2.11

6.23.0

com.typesafe

config

1.2.1

com.typesafe.scala-logging

scala-logging-api_2.11

2.1.2

com.typesafe.scala-logging

scala-logging-slf4j_2.11

2.1.2

com.univocity

univocity-parsers

2.5.9

com.vlkan

flatbuffers

1.2.0-3f79e055

com.zaxxer

HikariCP

2.4.1

commons-beanutils

commons-beanutils

1.7.0

commons-beanutils

commons-beanutils-core

1.8.0

commons-cli

commons-cli

1.2

commons-codec

commons-codec

1.10

commons-collections

commons-collections

3.2.2

commons-configuration

commons-configuration

1.6

commons-dbcp

commons-dbcp

1.4

commons-digester

commons-digester

1.8

commons-httpclient

commons-httpclient

3.1

commons-io

commons-io

2.4

commons-lang

commons-lang

2.6

commons-logging

commons-logging

1.1.3

commons-net

commons-net

2.2

commons-pool

commons-pool

1.5.4

info.ganglia.gmetric4j

gmetric4j

1.0.7

io.airlift

aircompressor

0.8

io.dropwizard.metrics

metrics-core

3.1.5

io.dropwizard.metrics

metrics-ganglia

3.1.5

io.dropwizard.metrics

metrics-graphite

3.1.5

io.dropwizard.metrics

metrics-healthchecks

3.1.5

io.dropwizard.metrics

metrics-jetty9

3.1.5

io.dropwizard.metrics

metrics-json

3.1.5

io.dropwizard.metrics

metrics-jvm

3.1.5

io.dropwizard.metrics

metrics-log4j

3.1.5

io.dropwizard.metrics

metrics-servlets

3.1.5

io.netty

netty

3.9.9.Final

io.netty

netty-all

4.1.17.Final

io.prometheus

simpleclient

0.0.16

io.prometheus

simpleclient_common

0.0.16

io.prometheus

simpleclient_dropwizard

0.0.16

io.prometheus

simpleclient_servlet

0.0.16

io.prometheus.jmx

collector

0.7

javax.activation

activation

1.1.1

javax.annotation

javax.annotation-api

1.2

javax.el

javax.el-api

2.2.4

javax.jdo

jdo-api

3.0.1

javax.servlet

javax.servlet-api

3.1.0

javax.servlet.jsp

jsp-api

2.1

javax.transaction

jta

1.1

javax.validation

validation-api

1.1.0.Final

javax.ws.rs

javax.ws.rs-api

2.0.1

javax.xml.bind

jaxb-api

2.2.2

javax.xml.stream

stax-api

1.0-2

javolution

javolution

5.5.1

jline

jline

2.11

joda-time

joda-time

2.9.3

log4j

apache-log4j-extras

1.2.17

log4j

log4j

1.2.17

net.hydromatic

eigenbase-properties

1.1.5

net.iharder

base64

2.3.8

net.java.dev.jets3t

jets3t

0.9.4

net.razorvine

pyrolite

4.13

net.sf.jpam

jpam

1.1

net.sf.opencsv

opencsv

2.3

net.sf.supercsv

super-csv

2.2.0

net.sourceforge.f2j

arpack_combined_all

0.1

org.acplt

oncrpc

1.0.7

org.antlr

ST4

4.0.4

org.antlr

antlr-runtime

3.4

org.antlr

antlr4-runtime

4.7

org.antlr

stringtemplate

3.2.1

org.apache.ant

ant

1.9.2

org.apache.ant

ant-jsch

1.9.2

org.apache.ant

ant-launcher

1.9.2

org.apache.arrow

arrow-format

0.8.0

org.apache.arrow

arrow-memory

0.8.0

org.apache.arrow

arrow-vector

0.8.0

org.apache.avro

avro

1.7.7

org.apache.avro

avro-ipc

1.7.7

org.apache.avro

avro-ipc-tests

1.7.7

org.apache.avro

avro-mapred-hadoop2

1.7.7

org.apache.calcite

calcite-avatica

1.2.0-incubating

org.apache.calcite

calcite-core

1.2.0-incubating

org.apache.calcite

calcite-linq4j

1.2.0-incubating

org.apache.commons

commons-compress

1.4.1

org.apache.commons

commons-crypto

1.0.0

org.apache.commons

commons-lang3

3.5

org.apache.commons

commons-math3

3.4.1

org.apache.curator

curator-client

2.7.1

org.apache.curator

curator-framework

2.7.1

org.apache.curator

curator-recipes

2.7.1

org.apache.derby

derby

10.12.1.1

org.apache.directory.api

api-asn1-api

1.0.0-M20

org.apache.directory.api

api-util

1.0.0-M20

org.apache.directory.server

apacheds-i18n

2.0.0-M15

org.apache.directory.server

apacheds-kerberos-codec

2.0.0-M15

org.apache.hadoop

hadoop-annotations

2.7.3

org.apache.hadoop

hadoop-auth

2.7.3

org.apache.hadoop

hadoop-client

2.7.3

org.apache.hadoop

hadoop-common

2.7.3

org.apache.hadoop

hadoop-hdfs

2.7.3

org.apache.hadoop

hadoop-mapreduce-client-app

2.7.3

org.apache.hadoop

hadoop-mapreduce-client-common

2.7.3

org.apache.hadoop

hadoop-mapreduce-client-core

2.7.3

org.apache.hadoop

hadoop-mapreduce-client-jobclient

2.7.3

org.apache.hadoop

hadoop-mapreduce-client-shuffle

2.7.3

org.apache.hadoop

hadoop-yarn-api

2.7.3

org.apache.hadoop

hadoop-yarn-client

2.7.3

org.apache.hadoop

hadoop-yarn-common

2.7.3

org.apache.hadoop

hadoop-yarn-server-common

2.7.3

org.apache.htrace

htrace-core

3.1.0-incubating

org.apache.httpcomponents

httpclient

4.5.4

org.apache.httpcomponents

httpcore

4.4.8

org.apache.ivy

ivy

2.4.0

org.apache.orc

orc-core-nohive

1.4.1

org.apache.orc

orc-mapreduce-nohive

1.4.1

org.apache.parquet

parquet-column

1.8.2-databricks1

org.apache.parquet

parquet-common

1.8.2-databricks1

org.apache.parquet

parquet-encoding

1.8.2-databricks1

org.apache.parquet

parquet-format

2.3.1

org.apache.parquet

parquet-hadoop

1.8.2-databricks1

org.apache.parquet

parquet-jackson

1.8.2-databricks1

org.apache.thrift

libfb303

0.9.3

org.apache.thrift

libthrift

0.9.3

org.apache.xbean

xbean-asm5-shaded

4.4

org.apache.zookeeper

zookeeper

3.4.6

org.bouncycastle

bcprov-jdk15on

1.58

org.codehaus.jackson

jackson-core-asl

1.9.13

org.codehaus.jackson

jackson-jaxrs

1.9.13

org.codehaus.jackson

jackson-mapper-asl

1.9.13

org.codehaus.jackson

jackson-xc

1.9.13

org.codehaus.janino

commons-compiler

3.0.8

org.codehaus.janino

janino

3.0.8

org.datanucleus

datanucleus-api-jdo

3.2.6

org.datanucleus

datanucleus-core

3.2.10

org.datanucleus

datanucleus-rdbms

3.2.9

org.eclipse.jetty

jetty-client

9.3.20.v20170531

org.eclipse.jetty

jetty-continuation

9.3.20.v20170531

org.eclipse.jetty

jetty-http

9.3.20.v20170531

org.eclipse.jetty

jetty-io

9.3.20.v20170531

org.eclipse.jetty

jetty-jndi

9.3.20.v20170531

org.eclipse.jetty

jetty-plus

9.3.20.v20170531

org.eclipse.jetty

jetty-proxy

9.3.20.v20170531

org.eclipse.jetty

jetty-security

9.3.20.v20170531

org.eclipse.jetty

jetty-server

9.3.20.v20170531

org.eclipse.jetty

jetty-servlet

9.3.20.v20170531

org.eclipse.jetty

jetty-servlets

9.3.20.v20170531

org.eclipse.jetty

jetty-util

9.3.20.v20170531

org.eclipse.jetty

jetty-webapp

9.3.20.v20170531

org.eclipse.jetty

jetty-xml

9.3.20.v20170531

org.fusesource.leveldbjni

leveldbjni-all

1.8

org.glassfish.hk2

hk2-api

2.4.0-b34

org.glassfish.hk2

hk2-locator

2.4.0-b34

org.glassfish.hk2

hk2-utils

2.4.0-b34

org.glassfish.hk2

osgi-resource-locator

1.0.1

org.glassfish.hk2.external

aopalliance-repackaged

2.4.0-b34

org.glassfish.hk2.external

javax.inject

2.4.0-b34

org.glassfish.jersey.bundles.repackaged

jersey-guava

2.22.2

org.glassfish.jersey.containers

jersey-container-servlet

2.22.2

org.glassfish.jersey.containers

jersey-container-servlet-core

2.22.2

org.glassfish.jersey.core

jersey-client

2.22.2

org.glassfish.jersey.core

jersey-common

2.22.2

org.glassfish.jersey.core

jersey-server

2.22.2

org.glassfish.jersey.media

jersey-media-jaxb

2.22.2

org.hibernate

hibernate-validator

5.1.1.Final

org.iq80.snappy

snappy

0.2

org.javassist

javassist

3.18.1-GA

org.jboss.logging

jboss-logging

3.1.3.GA

org.jdbi

jdbi

2.63.1

org.joda

joda-convert

1.7

org.jodd

jodd-core

3.5.2

org.json4s

json4s-ast_2.11

3.2.11

org.json4s

json4s-core_2.11

3.2.11

org.json4s

json4s-jackson_2.11

3.2.11

org.lz4

lz4-java

1.4.0

org.mariadb.jdbc

mariadb-java-client

2.1.2

org.mockito

mockito-all

1.9.5

org.objenesis

objenesis

2.1

org.postgresql

postgresql

42.1.4

org.roaringbitmap

RoaringBitmap

0.5.11

org.rocksdb

rocksdbjni

5.2.1

org.rosuda.REngine

REngine

2.1.0

org.scala-lang

scala-compiler_2.11

2.11.8

org.scala-lang

scala-library_2.11

2.11.8

org.scala-lang

scala-reflect_2.11

2.11.8

org.scala-lang

scalap_2.11

2.11.8

org.scala-lang.modules

scala-parser-combinators_2.11

1.0.2

org.scala-lang.modules

scala-xml_2.11

1.0.5

org.scala-sbt

test-interface

1.0

org.scalacheck

scalacheck_2.11

1.12.5

org.scalanlp

breeze-macros_2.11

0.13.2

org.scalanlp

breeze_2.11

0.13.2

org.scalatest

scalatest_2.11

2.2.6

org.slf4j

jcl-over-slf4j

1.7.16

org.slf4j

jul-to-slf4j

1.7.16

org.slf4j

slf4j-api

1.7.16

org.slf4j

slf4j-log4j12

1.7.16

org.spark-project.hive

hive-beeline

1.2.1.spark2

org.spark-project.hive

hive-cli

1.2.1.spark2

org.spark-project.hive

hive-exec

1.2.1.spark2

org.spark-project.hive

hive-jdbc

1.2.1.spark2

org.spark-project.hive

hive-metastore

1.2.1.spark2

org.spark-project.spark

unused

1.0.0

org.spire-math

spire-macros_2.11

0.13.0

org.spire-math

spire_2.11

0.13.0

org.springframework

spring-core

4.1.4.RELEASE

org.springframework

spring-test

4.1.4.RELEASE

org.tukaani

xz

1.0

org.typelevel

machinist_2.11

0.6.1

org.typelevel

macro-compat_2.11

1.1.1

org.xerial

sqlite-jdbc

3.8.11.2

org.xerial.snappy

snappy-java

1.1.2.6

org.yaml

snakeyaml

1.16

oro

oro

2.0.8

software.amazon.ion

ion-java

1.0.2

stax

stax-api

1.0.1

xmlenc

xmlenc

0.52