Skip to content

Commit

Permalink
move to contribs
Browse files Browse the repository at this point in the history
  • Loading branch information
mrk-its committed May 31, 2021
1 parent be5cede commit 6cbb87d
Show file tree
Hide file tree
Showing 5 changed files with 16 additions and 12 deletions.
3 changes: 3 additions & 0 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,9 @@ lazy val contribs = (project in file("contribs"))
commonSettings,
scalaStyleSettings,
releaseSettings,
libraryDependencies ++= Seq(
"com.amazonaws" % "aws-java-sdk" % "1.7.4"
),
(mappings in (Compile, packageBin)) := (mappings in (Compile, packageBin)).value ++
listPythonFiles(baseDirectory.value.getParentFile / "python"),

Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (2020) The Delta Lake Project Authors.
* Copyright (2021) The Delta Lake Project Authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -14,7 +14,7 @@
* limitations under the License.
*/

package org.apache.spark.sql.delta.storage
package io.delta.storage

import java.io.FileNotFoundException
import java.net.URI
Expand All @@ -28,6 +28,7 @@ import org.apache.hadoop.fs._
import org.apache.spark.SparkConf
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.util.FileNames
import org.apache.spark.sql.delta.storage.HadoopFileSystemLogStore

abstract class BaseExternalLogStore(sparkConf: SparkConf, hadoopConf: Configuration)
extends HadoopFileSystemLogStore(sparkConf, hadoopConf)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (2020) The Delta Lake Project Authors.
* Copyright (2021) The Delta Lake Project Authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -14,9 +14,10 @@
* limitations under the License.
*/

package org.apache.spark.sql.delta.storage
package io.delta.storage

import scala.collection.JavaConverters._
import scala.language.implicitConversions
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient
import com.amazonaws.auth.BasicAWSCredentials
import com.amazonaws.services.dynamodbv2.model.{
Expand All @@ -35,8 +36,6 @@ import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkConf
import com.amazonaws.regions.Region
import com.amazonaws.regions.Regions
import org.apache.spark.sql.delta.storage


/*
DynamoDB requirements:
Expand Down Expand Up @@ -126,7 +125,7 @@ class DynamoDBLogStore(
val length = item.get("length").getN.toLong
val modificationTime = item.get("modificationTime").getN.toLong
val isComplete = item.get("isComplete").getS() == "true"
storage.LogEntryMetadata(
LogEntryMetadata(
path = new Path(s"$parentPath/$filename"),
tempPath = tempPath,
length = length,
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (2020) The Delta Lake Project Authors.
* Copyright (2021) The Delta Lake Project Authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -14,7 +14,7 @@
* limitations under the License.
*/

package org.apache.spark.sql.delta.storage
package io.delta.storage

import java.util.concurrent.{ConcurrentHashMap, TimeUnit}

Expand Down
7 changes: 4 additions & 3 deletions examples/python/extra/dynamodb_logstore.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@
pass

# Create SparkContext
sc = SparkContext()
sqlContext = SQLContext(sc)
# sc = SparkContext()
# sqlContext = SQLContext(sc)

# Enable SQL for the current spark session. we need to set the following configs to enable SQL
# Commands
Expand All @@ -47,9 +47,10 @@
.master("local[*]") \
.config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") \
.config("spark.sql.sources.parallelPartitionDiscovery.parallelism", "8") \
.config("spark.delta.logStore.class", "org.apache.spark.sql.delta.storage.DynamoDBLogStore") \
.config("spark.delta.logStore.class", "io.delta.storage.DynamoDBLogStore") \
.getOrCreate()


# Apache Spark 2.4.x has a known issue (SPARK-25003) that requires explicit activation
# of the extension and cloning of the session. This will unnecessary in Apache Spark 3.x.
if spark.sparkContext.version < "3.":
Expand Down

0 comments on commit 6cbb87d

Please sign in to comment.