Refactor Copier.copyObject(AmazonS3.Client,Bucket,RemoteKey,MD5Hash,RemoteKey) (#137)

* [core] Convert HashService into an effect Hasher

* [core] LocalFileStreamSuite fix prep of test env

* [core] LocalFileStreamSuite Refactor

* [core] Add Hasher.hashObjectChunk

* [core] make MD5HashGenerator private

Moved MD5HashGenerator into package hasher and made it private to that
package, together with the Hasher effect which is the only permitted
user of it. Added hex and digest methods to Hasher.

Similar for the ETageGenerator in [storage-aws], moved into a hasher
package and made private. It can only be accessed using the S3Hasher
effect.

* [domain] Add Monoid

* [domain] Sources use Monoid

* [core] ActionGenerator Refactor

* [core] Use >>= and *> operators

* [config] Refactoring

* [storage-aws] Refactoring

* [domain] Refactoring

* [core] Refactoring

* [config] refactoring

* Refactoring

* [core] PlanBuilder Refactoring

* [core] Make PlanBuilder into an object

* [storate-aws] Copier Reduce the number of parameters
This commit is contained in:
Paul Campbell 2019-08-01 08:34:58 +01:00 committed by GitHub
parent a749b1f5b0
commit 899e0724c9
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
47 changed files with 519 additions and 540 deletions

View file

@ -122,7 +122,7 @@ lazy val config = (project in file("config"))
.settings(testDependencies) .settings(testDependencies)
.settings(commandLineParsing) .settings(commandLineParsing)
.settings(assemblyJarName in assembly := "config.jar") .settings(assemblyJarName in assembly := "config.jar")
.dependsOn(domain) .dependsOn(domain % "compile->compile;test->test")
.dependsOn(filesystem) .dependsOn(filesystem)
lazy val filesystem = (project in file("filesystem")) lazy val filesystem = (project in file("filesystem"))

View file

@ -4,6 +4,7 @@ import net.kemitix.thorp.config.Config
import net.kemitix.thorp.console.Console import net.kemitix.thorp.console.Console
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import net.kemitix.thorp.storage.aws.S3Storage import net.kemitix.thorp.storage.aws.S3Storage
import net.kemitix.thorp.storage.aws.hasher.S3Hasher
import zio.{App, ZIO} import zio.{App, ZIO}
object Main extends App { object Main extends App {
@ -13,6 +14,7 @@ object Main extends App {
with Console.Live with Console.Live
with Config.Live with Config.Live
with FileSystem.Live with FileSystem.Live
with S3Hasher.Live
override def run(args: List[String]): ZIO[Environment, Nothing, Int] = override def run(args: List[String]): ZIO[Environment, Nothing, Int] =
Program Program

View file

@ -5,7 +5,6 @@ import net.kemitix.thorp.console._
import net.kemitix.thorp.core.CoreTypes.CoreProgram import net.kemitix.thorp.core.CoreTypes.CoreProgram
import net.kemitix.thorp.core._ import net.kemitix.thorp.core._
import net.kemitix.thorp.domain.StorageQueueEvent import net.kemitix.thorp.domain.StorageQueueEvent
import net.kemitix.thorp.storage.aws.S3HashService.defaultHashService
import zio.ZIO import zio.ZIO
trait Program { trait Program {
@ -27,7 +26,7 @@ trait Program {
private def execute = { private def execute = {
for { for {
plan <- PlanBuilder.createPlan(defaultHashService) plan <- PlanBuilder.createPlan
archive <- UnversionedMirrorArchive.default(plan.syncTotals) archive <- UnversionedMirrorArchive.default(plan.syncTotals)
events <- applyPlan(archive, plan) events <- applyPlan(archive, plan)
_ <- SyncLogging.logRunFinished(events) _ <- SyncLogging.logRunFinished(events)
@ -39,7 +38,7 @@ trait Program {
_ <- Console.putStrLn("There were errors:") _ <- Console.putStrLn("There were errors:")
_ <- throwable match { _ <- throwable match {
case ConfigValidationException(errors) => case ConfigValidationException(errors) =>
ZIO.foreach(errors)(error => Console.putStrLn(s"- $error")) ZIO.foreach_(errors)(error => Console.putStrLn(s"- $error"))
} }
} yield () } yield ()

View file

@ -14,7 +14,7 @@ object ConfigOption {
case class Source(path: Path) extends ConfigOption { case class Source(path: Path) extends ConfigOption {
override def update(config: Configuration): Configuration = override def update(config: Configuration): Configuration =
sources.modify(_ ++ path)(config) sources.modify(_ + path)(config)
} }
case class Bucket(name: String) extends ConfigOption { case class Bucket(name: String) extends ConfigOption {

View file

@ -1,5 +1,5 @@
package net.kemitix.thorp.config package net.kemitix.thorp.config
final case class ConfigValidationException( final case class ConfigValidationException(
errors: List[ConfigValidation] errors: Seq[ConfigValidation]
) extends Exception ) extends Exception

View file

@ -22,17 +22,15 @@ sealed trait ConfigValidator {
def validateSources( def validateSources(
sources: Sources): Either[List[ConfigValidation], Sources] = sources: Sources): Either[List[ConfigValidation], Sources] =
(for { sources.paths.foldLeft(List[ConfigValidation]()) {
x <- sources.paths.foldLeft(List[ConfigValidation]()) { (acc: List[ConfigValidation], path) =>
(acc: List[ConfigValidation], path) => {
{ validateSource(path) match {
validateSource(path) match { case Left(errors) => acc ++ errors
case Left(errors) => acc ++ errors case Right(_) => acc
case Right(_) => acc
}
} }
} }
} yield x) match { } match {
case Nil => Right(sources) case Nil => Right(sources)
case errors => Left(errors) case errors => Left(errors)
} }

View file

@ -17,10 +17,8 @@ trait ConfigurationBuilder {
def buildConfig(priorityOpts: ConfigOptions) def buildConfig(priorityOpts: ConfigOptions)
: ZIO[FileSystem, ConfigValidationException, Configuration] = : ZIO[FileSystem, ConfigValidationException, Configuration] =
(for { (getConfigOptions(priorityOpts).map(collateOptions) >>=
config <- getConfigOptions(priorityOpts).map(collateOptions) ConfigValidator.validateConfig)
valid <- ConfigValidator.validateConfig(config)
} yield valid)
.catchAll(errors => ZIO.fail(ConfigValidationException(errors))) .catchAll(errors => ZIO.fail(ConfigValidationException(errors)))
private def getConfigOptions(priorityOpts: ConfigOptions) = private def getConfigOptions(priorityOpts: ConfigOptions) =

View file

@ -3,24 +3,25 @@ package net.kemitix.thorp.config
import java.nio.file.Path import java.nio.file.Path
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import zio.{IO, ZIO} import zio.{IO, TaskR, ZIO}
trait ParseConfigFile { trait ParseConfigFile {
def parseFile( def parseFile(
filename: Path): ZIO[FileSystem, List[ConfigValidation], ConfigOptions] = filename: Path): ZIO[FileSystem, Seq[ConfigValidation], ConfigOptions] =
readFile(filename) (readFile(filename) >>= ParseConfigLines.parseLines)
.map(ParseConfigLines.parseLines) .catchAll(
.catchAll(h => h =>
IO.fail( IO.fail(
List(ConfigValidation.ErrorReadingFile(filename, h.getMessage)))) List(ConfigValidation.ErrorReadingFile(filename, h.getMessage))))
private def readFile(filename: Path) = private def readFile(filename: Path) =
FileSystem FileSystem.exists(filename.toFile) >>= readLines(filename)
.exists(filename.toFile)
.flatMap( private def readLines(filename: Path)(
if (_) FileSystem.lines(filename.toFile) exists: Boolean): TaskR[FileSystem, Seq[String]] =
else ZIO.succeed(List.empty)) if (exists) FileSystem.lines(filename.toFile)
else ZIO.succeed(Seq.empty)
} }

View file

@ -3,22 +3,16 @@ package net.kemitix.thorp.config
import java.nio.file.Paths import java.nio.file.Paths
import java.util.regex.Pattern import java.util.regex.Pattern
import net.kemitix.thorp.config.ConfigOption.{ import net.kemitix.thorp.config.ConfigOption._
Bucket, import zio.UIO
Debug,
Exclude,
Include,
Prefix,
Source
}
trait ParseConfigLines { trait ParseConfigLines {
private val pattern = "^\\s*(?<key>\\S*)\\s*=\\s*(?<value>\\S*)\\s*$" private val pattern = "^\\s*(?<key>\\S*)\\s*=\\s*(?<value>\\S*)\\s*$"
private val format = Pattern.compile(pattern) private val format = Pattern.compile(pattern)
def parseLines(lines: List[String]): ConfigOptions = def parseLines(lines: Seq[String]): UIO[ConfigOptions] =
ConfigOptions(lines.flatMap(parseLine)) UIO(ConfigOptions(lines.flatMap(parseLine).toList))
private def parseLine(str: String) = private def parseLine(str: String) =
format.matcher(str) match { format.matcher(str) match {
@ -40,7 +34,7 @@ trait ParseConfigLines {
case _ => None case _ => None
} }
def truthy(value: String): Boolean = private def truthy(value: String): Boolean =
value.toLowerCase match { value.toLowerCase match {
case "true" => true case "true" => true
case "yes" => true case "yes" => true

View file

@ -8,24 +8,16 @@ trait SourceConfigLoader {
val thorpConfigFileName = ".thorp.conf" val thorpConfigFileName = ".thorp.conf"
def loadSourceConfigs def loadSourceConfigs(
: Sources => ZIO[FileSystem, List[ConfigValidation], ConfigOptions] = sources: Sources): ZIO[FileSystem, Seq[ConfigValidation], ConfigOptions] =
sources => { ZIO
.foreach(sources.paths) { path =>
val sourceConfigOptions = ParseConfigFile.parseFile(path.resolve(thorpConfigFileName))
ConfigOptions(sources.paths.map(ConfigOption.Source)) }
.map(_.foldLeft(ConfigOptions(sources.paths.map(ConfigOption.Source))) {
val reduce: List[ConfigOptions] => ConfigOptions = (acc, co) =>
_.foldLeft(sourceConfigOptions) { (acc, co) =>
acc ++ co acc ++ co
} })
ZIO
.foreach(sources.paths) { path =>
ParseConfigFile.parseFile(path.resolve(thorpConfigFileName))
}
.map(reduce)
}
} }

View file

@ -1,12 +1,6 @@
package net.kemitix.thorp.core package net.kemitix.thorp.config
import net.kemitix.thorp.config.{ import net.kemitix.thorp.domain.{Sources, TemporaryFolder}
ConfigOption,
ConfigOptions,
ConfigQuery,
ConfigurationBuilder
}
import net.kemitix.thorp.domain.Sources
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import org.scalatest.FunSpec import org.scalatest.FunSpec
import zio.DefaultRuntime import zio.DefaultRuntime

View file

@ -1,8 +1,7 @@
package net.kemitix.thorp.core package net.kemitix.thorp.config
import java.nio.file.Paths import java.nio.file.Paths
import net.kemitix.thorp.config.{ConfigOption, ConfigOptions, ConfigQuery}
import net.kemitix.thorp.domain.Sources import net.kemitix.thorp.domain.Sources
import org.scalatest.FreeSpec import org.scalatest.FreeSpec

View file

@ -1,12 +1,7 @@
package net.kemitix.thorp.core package net.kemitix.thorp.config
import java.nio.file.{Path, Paths} import java.nio.file.{Path, Paths}
import net.kemitix.thorp.config.{
ConfigOption,
ConfigOptions,
ConfigurationBuilder
}
import net.kemitix.thorp.domain.Filter.{Exclude, Include} import net.kemitix.thorp.domain.Filter.{Exclude, Include}
import net.kemitix.thorp.domain._ import net.kemitix.thorp.domain._
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem

View file

@ -1,13 +1,7 @@
package net.kemitix.thorp.core package net.kemitix.thorp.config
import java.nio.file.{Path, Paths} import java.nio.file.{Path, Paths}
import net.kemitix.thorp.config.{
ConfigOption,
ConfigOptions,
ParseConfigFile,
Resource
}
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import org.scalatest.FunSpec import org.scalatest.FunSpec
import zio.DefaultRuntime import zio.DefaultRuntime

View file

@ -0,0 +1,88 @@
package net.kemitix.thorp.config
import java.nio.file.Paths
import org.scalatest.FunSpec
import zio.DefaultRuntime
class ParseConfigLinesTest extends FunSpec {
describe("parse single lines") {
describe("source") {
it("should parse") {
val expected =
Right(
ConfigOptions(
List(ConfigOption.Source(Paths.get("/path/to/source")))))
val result = invoke(List("source = /path/to/source"))
assertResult(expected)(result)
}
}
describe("bucket") {
it("should parse") {
val expected =
Right(ConfigOptions(List(ConfigOption.Bucket("bucket-name"))))
val result = invoke(List("bucket = bucket-name"))
assertResult(expected)(result)
}
}
describe("prefix") {
it("should parse") {
val expected =
Right(ConfigOptions(List(ConfigOption.Prefix("prefix/to/files"))))
val result = invoke(List("prefix = prefix/to/files"))
assertResult(expected)(result)
}
}
describe("include") {
it("should parse") {
val expected =
Right(ConfigOptions(List(ConfigOption.Include("path/to/include"))))
val result = invoke(List("include = path/to/include"))
assertResult(expected)(result)
}
}
describe("exclude") {
it("should parse") {
val expected =
Right(ConfigOptions(List(ConfigOption.Exclude("path/to/exclude"))))
val result = invoke(List("exclude = path/to/exclude"))
assertResult(expected)(result)
}
}
describe("debug - true") {
it("should parse") {
val expected = Right(ConfigOptions(List(ConfigOption.Debug())))
val result = invoke(List("debug = true"))
assertResult(expected)(result)
}
}
describe("debug - false") {
it("should parse") {
val expected = Right(ConfigOptions())
val result = invoke(List("debug = false"))
assertResult(expected)(result)
}
}
describe("comment line") {
it("should be ignored") {
val expected = Right(ConfigOptions())
val result = invoke(List("# ignore me"))
assertResult(expected)(result)
}
}
describe("unrecognised option") {
it("should be ignored") {
val expected = Right(ConfigOptions())
val result = invoke(List("unsupported = option"))
assertResult(expected)(result)
}
}
def invoke(lines: List[String]) = {
new DefaultRuntime {}.unsafeRunSync {
ParseConfigLines.parseLines(lines)
}.toEither
}
}
}

View file

@ -13,35 +13,37 @@ object ActionGenerator {
): ZIO[Config, Nothing, Stream[Action]] = ): ZIO[Config, Nothing, Stream[Action]] =
for { for {
bucket <- Config.bucket bucket <- Config.bucket
} yield } yield genAction(s3MetaData, previousActions, bucket)
s3MetaData match {
// #1 local exists, remote exists, remote matches - do nothing private def genAction(s3MetaData: S3MetaData,
case S3MetaData(localFile, _, Some(RemoteMetaData(key, hash, _))) previousActions: Stream[Action],
if localFile.matches(hash) => bucket: Bucket): Stream[Action] = {
doNothing(bucket, key) s3MetaData match {
// #2 local exists, remote is missing, other matches - copy // #1 local exists, remote exists, remote matches - do nothing
case S3MetaData(localFile, matchByHash, None) if matchByHash.nonEmpty => case S3MetaData(localFile, _, Some(RemoteMetaData(key, hash, _)))
copyFile(bucket, localFile, matchByHash) if localFile.matches(hash) =>
// #3 local exists, remote is missing, other no matches - upload doNothing(bucket, key)
case S3MetaData(localFile, matchByHash, None) // #2 local exists, remote is missing, other matches - copy
if matchByHash.isEmpty && case S3MetaData(localFile, matchByHash, None) if matchByHash.nonEmpty =>
isUploadAlreadyQueued(previousActions)(localFile) => copyFile(bucket, localFile, matchByHash)
uploadFile(bucket, localFile) // #3 local exists, remote is missing, other no matches - upload
// #4 local exists, remote exists, remote no match, other matches - copy case S3MetaData(localFile, matchByHash, None)
case S3MetaData(localFile, if matchByHash.isEmpty &&
matchByHash, isUploadAlreadyQueued(previousActions)(localFile) =>
Some(RemoteMetaData(_, hash, _))) uploadFile(bucket, localFile)
if !localFile.matches(hash) && // #4 local exists, remote exists, remote no match, other matches - copy
matchByHash.nonEmpty => case S3MetaData(localFile, matchByHash, Some(RemoteMetaData(_, hash, _)))
copyFile(bucket, localFile, matchByHash) if !localFile.matches(hash) &&
// #5 local exists, remote exists, remote no match, other no matches - upload matchByHash.nonEmpty =>
case S3MetaData(localFile, matchByHash, Some(_)) copyFile(bucket, localFile, matchByHash)
if matchByHash.isEmpty => // #5 local exists, remote exists, remote no match, other no matches - upload
uploadFile(bucket, localFile) case S3MetaData(localFile, matchByHash, Some(_)) if matchByHash.isEmpty =>
// fallback uploadFile(bucket, localFile)
case S3MetaData(localFile, _, _) => // fallback
doNothing(bucket, localFile.remoteKey) case S3MetaData(localFile, _, _) =>
} doNothing(bucket, localFile.remoteKey)
}
}
private def key = LocalFile.remoteKey ^|-> RemoteKey.key private def key = LocalFile.remoteKey ^|-> RemoteKey.key

View file

@ -2,13 +2,14 @@ package net.kemitix.thorp.core
import net.kemitix.thorp.config.Config import net.kemitix.thorp.config.Config
import net.kemitix.thorp.console.Console import net.kemitix.thorp.console.Console
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import net.kemitix.thorp.storage.api.Storage import net.kemitix.thorp.storage.api.Storage
import zio.ZIO import zio.ZIO
object CoreTypes { object CoreTypes {
type CoreEnv = Storage with Console with Config with FileSystem type CoreEnv = Storage with Console with Config with FileSystem with Hasher
type CoreProgram[A] = ZIO[CoreEnv, Throwable, A] type CoreProgram[A] = ZIO[CoreEnv, Throwable, A]
} }

View file

@ -1,16 +0,0 @@
package net.kemitix.thorp.core
import java.nio.file.Path
import net.kemitix.thorp.domain.{HashType, MD5Hash}
import net.kemitix.thorp.filesystem.FileSystem
import zio.TaskR
/**
* Creates one, or more, hashes for local objects.
*/
trait HashService {
def hashLocalObject(path: Path): TaskR[FileSystem, Map[HashType, MD5Hash]]
}

View file

@ -1,48 +1,43 @@
package net.kemitix.thorp.core package net.kemitix.thorp.core
import java.io.File
import java.nio.file.Path import java.nio.file.Path
import net.kemitix.thorp.config.Config import net.kemitix.thorp.config.Config
import net.kemitix.thorp.core.KeyGenerator.generateKey import net.kemitix.thorp.core.KeyGenerator.generateKey
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.domain._ import net.kemitix.thorp.domain._
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import zio.{Task, TaskR, ZIO} import zio.{Task, TaskR, ZIO}
object LocalFileStream { object LocalFileStream {
def findFiles(hashService: HashService)( def findFiles(
source: Path source: Path
): TaskR[Config with FileSystem, LocalFiles] = { ): TaskR[Config with FileSystem with Hasher, LocalFiles] = {
def recurseIntoSubDirectories( def recurseIntoSubDirectories(
path: Path): TaskR[Config with FileSystem, LocalFiles] = path: Path): TaskR[Config with FileSystem with Hasher, LocalFiles] =
path.toFile match { path.toFile match {
case f if f.isDirectory => loop(path) case f if f.isDirectory => loop(path)
case _ => pathToLocalFile(hashService)(path) case _ => localFile(path)
} }
def recurse( def recurse(paths: Stream[Path])
paths: Stream[Path]): TaskR[Config with FileSystem, LocalFiles] = : TaskR[Config with FileSystem with Hasher, LocalFiles] =
for { for {
recursed <- ZIO.foreach(paths)(path => recurseIntoSubDirectories(path)) recursed <- ZIO.foreach(paths)(path => recurseIntoSubDirectories(path))
} yield LocalFiles.reduce(recursed.toStream) } yield LocalFiles.reduce(recursed.toStream)
def loop(path: Path): TaskR[Config with FileSystem, LocalFiles] = { def loop(
path: Path): TaskR[Config with FileSystem with Hasher, LocalFiles] =
for { dirPaths(path) >>= recurse
paths <- dirPaths(path)
localFiles <- recurse(paths)
} yield localFiles
}
loop(source) loop(source)
} }
private def dirPaths(path: Path) = private def dirPaths(path: Path) =
for { listFiles(path) >>= includedDirPaths
paths <- listFiles(path)
filtered <- includedDirPaths(paths)
} yield filtered
private def includedDirPaths(paths: Stream[Path]) = private def includedDirPaths(paths: Stream[Path]) =
for { for {
@ -53,12 +48,12 @@ object LocalFileStream {
.filter({ case (_, included) => included }) .filter({ case (_, included) => included })
.map({ case (path, _) => path }) .map({ case (path, _) => path })
private def localFile(hashService: HashService)(path: Path) = { private def localFile(path: Path) = {
val file = path.toFile val file = path.toFile
for { for {
sources <- Config.sources sources <- Config.sources
prefix <- Config.prefix prefix <- Config.prefix
hash <- hashService.hashLocalObject(path) hash <- Hasher.hashObject(path)
localFile = LocalFile(file, localFile = LocalFile(file,
sources.forPath(path).toFile, sources.forPath(path).toFile,
hash, hash,
@ -72,16 +67,17 @@ object LocalFileStream {
private def listFiles(path: Path) = private def listFiles(path: Path) =
for { for {
files <- Task(path.toFile.listFiles) files <- Task(path.toFile.listFiles)
_ <- Task.when(files == null)( _ <- filesMustExist(path, files)
Task.fail(new IllegalArgumentException(s"Directory not found $path")))
} yield Stream(files: _*).map(_.toPath) } yield Stream(files: _*).map(_.toPath)
private def filesMustExist(path: Path, files: Array[File]) = {
Task.when(files == null)(
Task.fail(new IllegalArgumentException(s"Directory not found $path")))
}
private def isIncluded(path: Path) = private def isIncluded(path: Path) =
for { for {
filters <- Config.filters filters <- Config.filters
} yield Filter.isIncluded(filters)(path) } yield Filter.isIncluded(path)(filters)
private def pathToLocalFile(hashService: HashService)(path: Path) =
localFile(hashService)(path)
} }

View file

@ -3,30 +3,36 @@ package net.kemitix.thorp.core
import net.kemitix.thorp.config.Config import net.kemitix.thorp.config.Config
import net.kemitix.thorp.console._ import net.kemitix.thorp.console._
import net.kemitix.thorp.core.Action._ import net.kemitix.thorp.core.Action._
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.domain._ import net.kemitix.thorp.domain._
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import net.kemitix.thorp.storage.api.Storage import net.kemitix.thorp.storage.api.Storage
import zio.{TaskR, ZIO} import zio.{TaskR, ZIO}
trait PlanBuilder { object PlanBuilder {
def createPlan(hashService: HashService) def createPlan
: TaskR[Storage with Console with Config with FileSystem, SyncPlan] = : TaskR[Storage with Console with Config with FileSystem with Hasher,
for { SyncPlan] =
_ <- SyncLogging.logRunStart SyncLogging.logRunStart *> buildPlan
actions <- buildPlan(hashService)
} yield actions
private def buildPlan(hashService: HashService) = private def buildPlan =
gatherMetadata >>= assemblePlan
private def gatherMetadata =
fetchRemoteData &&& findLocalFiles
private def fetchRemoteData =
for { for {
metadata <- gatherMetadata(hashService) bucket <- Config.bucket
plan <- assemblePlan(metadata) prefix <- Config.prefix
} yield plan objects <- Storage.list(bucket, prefix)
} yield objects
private def assemblePlan(metadata: (S3ObjectsData, LocalFiles)) = private def assemblePlan(metadata: (S3ObjectsData, LocalFiles)) =
metadata match { metadata match {
case (remoteData, localData) => case (remoteData, localData) =>
createActions(remoteData, localData) createActions(remoteData, localData.localFiles)
.map(_.filter(doesSomething).sortBy(SequencePlan.order)) .map(_.filter(doesSomething).sortBy(SequencePlan.order))
.map( .map(
SyncPlan(_, SyncTotals(localData.count, localData.totalSizeBytes))) SyncPlan(_, SyncTotals(localData.count, localData.totalSizeBytes)))
@ -34,11 +40,11 @@ trait PlanBuilder {
private def createActions( private def createActions(
remoteData: S3ObjectsData, remoteData: S3ObjectsData,
localData: LocalFiles localFiles: Stream[LocalFile]
) = ) =
for { for {
fileActions <- actionsForLocalFiles(remoteData, localData) fileActions <- actionsForLocalFiles(remoteData, localFiles)
remoteActions <- actionsForRemoteKeys(remoteData) remoteActions <- actionsForRemoteKeys(remoteData.byKey.keys)
} yield fileActions ++ remoteActions } yield fileActions ++ remoteActions
private def doesSomething: Action => Boolean = { private def doesSomething: Action => Boolean = {
@ -48,12 +54,10 @@ trait PlanBuilder {
private def actionsForLocalFiles( private def actionsForLocalFiles(
remoteData: S3ObjectsData, remoteData: S3ObjectsData,
localData: LocalFiles localFiles: Stream[LocalFile]
) = ) =
ZIO.foldLeft(localData.localFiles)(Stream.empty[Action])( ZIO.foldLeft(localFiles)(Stream.empty[Action])((acc, localFile) =>
(acc, localFile) => createActionFromLocalFile(remoteData, acc, localFile).map(_ ++ acc))
createActionFromLocalFile(remoteData, acc, localFile)
.map(actions => actions ++ acc))
private def createActionFromLocalFile( private def createActionFromLocalFile(
remoteData: S3ObjectsData, remoteData: S3ObjectsData,
@ -64,11 +68,9 @@ trait PlanBuilder {
S3MetaDataEnricher.getMetadata(localFile, remoteData), S3MetaDataEnricher.getMetadata(localFile, remoteData),
previousActions) previousActions)
private def actionsForRemoteKeys(remoteData: S3ObjectsData) = private def actionsForRemoteKeys(remoteKeys: Iterable[RemoteKey]) =
ZIO.foldLeft(remoteData.byKey.keys)(Stream.empty[Action]) { ZIO.foldLeft(remoteKeys)(Stream.empty[Action])((acc, remoteKey) =>
(acc, remoteKey) => createActionFromRemoteKey(remoteKey).map(_ #:: acc))
createActionFromRemoteKey(remoteKey).map(action => action #:: acc)
}
private def createActionFromRemoteKey(remoteKey: RemoteKey) = private def createActionFromRemoteKey(remoteKey: RemoteKey) =
for { for {
@ -80,33 +82,13 @@ trait PlanBuilder {
if (needsDeleted) ToDelete(bucket, remoteKey, 0L) if (needsDeleted) ToDelete(bucket, remoteKey, 0L)
else DoNothing(bucket, remoteKey, 0L) else DoNothing(bucket, remoteKey, 0L)
private def gatherMetadata(hashService: HashService) = private def findLocalFiles =
for { SyncLogging.logFileScan *> findFiles
remoteData <- fetchRemoteData
localData <- findLocalFiles(hashService)
} yield (remoteData, localData)
private def fetchRemoteData = private def findFiles =
for {
bucket <- Config.bucket
prefix <- Config.prefix
objects <- Storage.list(bucket, prefix)
} yield objects
private def findLocalFiles(hashService: HashService) =
for {
_ <- SyncLogging.logFileScan
localFiles <- findFiles(hashService)
} yield localFiles
private def findFiles(hashService: HashService) =
for { for {
sources <- Config.sources sources <- Config.sources
paths = sources.paths found <- ZIO.foreach(sources.paths)(LocalFileStream.findFiles)
found <- ZIO.foreach(paths)(path =>
LocalFileStream.findFiles(hashService)(path))
} yield LocalFiles.reduce(found.toStream) } yield LocalFiles.reduce(found.toStream)
} }
object PlanBuilder extends PlanBuilder

View file

@ -1,19 +0,0 @@
package net.kemitix.thorp.core
import java.nio.file.Path
import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.domain.{HashType, MD5Hash}
import net.kemitix.thorp.filesystem.FileSystem
import zio.TaskR
case class SimpleHashService() extends HashService {
override def hashLocalObject(
path: Path
): TaskR[FileSystem, Map[HashType, MD5Hash]] =
for {
md5 <- MD5HashGenerator.md5File(path)
} yield Map(MD5 -> md5)
}

View file

@ -33,13 +33,11 @@ trait SyncLogging {
actions: Stream[StorageQueueEvent] actions: Stream[StorageQueueEvent]
): ZIO[Console, Nothing, Unit] = { ): ZIO[Console, Nothing, Unit] = {
val counters = actions.foldLeft(Counters())(countActivities) val counters = actions.foldLeft(Counters())(countActivities)
for { Console.putStrLn(eraseToEndOfScreen) *>
_ <- Console.putStrLn(eraseToEndOfScreen) Console.putStrLn(s"Uploaded ${counters.uploaded} files") *>
_ <- Console.putStrLn(s"Uploaded ${counters.uploaded} files") Console.putStrLn(s"Copied ${counters.copied} files") *>
_ <- Console.putStrLn(s"Copied ${counters.copied} files") Console.putStrLn(s"Deleted ${counters.deleted} files") *>
_ <- Console.putStrLn(s"Deleted ${counters.deleted} files") Console.putStrLn(s"Errors ${counters.errors}")
_ <- Console.putStrLn(s"Errors ${counters.errors}")
} yield ()
} }
private def countActivities: (Counters, StorageQueueEvent) => Counters = private def countActivities: (Counters, StorageQueueEvent) => Counters =

View file

@ -27,7 +27,7 @@ trait ThorpArchive {
def logEvent( def logEvent(
event: StorageQueueEvent event: StorageQueueEvent
): TaskR[Console with Config, Unit] = ): TaskR[Console with Config, StorageQueueEvent] =
event match { event match {
case UploadQueueEvent(remoteKey, _) => case UploadQueueEvent(remoteKey, _) =>
for { for {
@ -37,7 +37,7 @@ trait ThorpArchive {
_ <- TaskR.when(!batchMode)( _ <- TaskR.when(!batchMode)(
Console.putStrLn( Console.putStrLn(
s"${GREEN}Uploaded:$RESET ${remoteKey.key}$eraseToEndOfScreen")) s"${GREEN}Uploaded:$RESET ${remoteKey.key}$eraseToEndOfScreen"))
} yield () } yield event
case CopyQueueEvent(sourceKey, targetKey) => case CopyQueueEvent(sourceKey, targetKey) =>
for { for {
batchMode <- Config.batchMode batchMode <- Config.batchMode
@ -47,7 +47,7 @@ trait ThorpArchive {
Console.putStrLn( Console.putStrLn(
s"${GREEN}Copied:$RESET ${sourceKey.key} => ${targetKey.key}$eraseToEndOfScreen") s"${GREEN}Copied:$RESET ${sourceKey.key} => ${targetKey.key}$eraseToEndOfScreen")
) )
} yield () } yield event
case DeleteQueueEvent(remoteKey) => case DeleteQueueEvent(remoteKey) =>
for { for {
batchMode <- Config.batchMode batchMode <- Config.batchMode
@ -55,7 +55,7 @@ trait ThorpArchive {
_ <- TaskR.when(!batchMode)( _ <- TaskR.when(!batchMode)(
Console.putStrLn( Console.putStrLn(
s"${GREEN}Deleted:$RESET ${remoteKey.key}$eraseToEndOfScreen")) s"${GREEN}Deleted:$RESET ${remoteKey.key}$eraseToEndOfScreen"))
} yield () } yield event
case ErrorQueueEvent(action, _, e) => case ErrorQueueEvent(action, _, e) =>
for { for {
batchMode <- Config.batchMode batchMode <- Config.batchMode
@ -64,9 +64,9 @@ trait ThorpArchive {
s"${action.name} failed: ${action.keys}: ${e.getMessage}")) s"${action.name} failed: ${action.keys}: ${e.getMessage}"))
_ <- TaskR.when(!batchMode)(Console.putStrLn( _ <- TaskR.when(!batchMode)(Console.putStrLn(
s"${RED}ERROR:$RESET ${action.name} ${action.keys}: ${e.getMessage}$eraseToEndOfScreen")) s"${RED}ERROR:$RESET ${action.name} ${action.keys}: ${e.getMessage}$eraseToEndOfScreen"))
} yield () } yield event
case DoNothingQueueEvent(_) => TaskR(()) case DoNothingQueueEvent(_) => TaskR(event)
case ShutdownQueueEvent() => TaskR(()) case ShutdownQueueEvent() => TaskR(event)
} }
} }

View file

@ -18,20 +18,11 @@ case class UnversionedMirrorArchive(syncTotals: SyncTotals)
): TaskR[Storage with Console with Config, StorageQueueEvent] = ): TaskR[Storage with Console with Config, StorageQueueEvent] =
action match { action match {
case ToUpload(bucket, localFile, _) => case ToUpload(bucket, localFile, _) =>
for { doUpload(index, totalBytesSoFar, bucket, localFile) >>= logEvent
event <- doUpload(index, totalBytesSoFar, bucket, localFile)
_ <- logEvent(event)
} yield event
case ToCopy(bucket, sourceKey, hash, targetKey, _) => case ToCopy(bucket, sourceKey, hash, targetKey, _) =>
for { Storage.copy(bucket, sourceKey, hash, targetKey) >>= logEvent
event <- Storage.copy(bucket, sourceKey, hash, targetKey)
_ <- logEvent(event)
} yield event
case ToDelete(bucket, remoteKey, _) => case ToDelete(bucket, remoteKey, _) =>
for { Storage.delete(bucket, remoteKey) >>= logEvent
event <- Storage.delete(bucket, remoteKey)
_ <- logEvent(event)
} yield event
case DoNothing(_, remoteKey, _) => case DoNothing(_, remoteKey, _) =>
Task(DoNothingQueueEvent(remoteKey)) Task(DoNothingQueueEvent(remoteKey))
} }

View file

@ -0,0 +1,96 @@
package net.kemitix.thorp.core.hasher
import java.nio.file.Path
import java.util.concurrent.atomic.AtomicReference
import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.domain.{HashType, MD5Hash}
import net.kemitix.thorp.filesystem.FileSystem
import zio.{TaskR, ZIO}
/**
* Creates one, or more, hashes for local objects.
*/
trait Hasher {
val hasher: Hasher.Service
}
object Hasher {
trait Service {
def hashObject(
path: Path): TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]]
def hashObjectChunk(
path: Path,
chunkNumber: Long,
chunkSize: Long): TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]]
def hex(in: Array[Byte]): TaskR[Hasher, String]
def digest(in: String): TaskR[Hasher, Array[Byte]]
}
trait Live extends Hasher {
val hasher: Service = new Service {
override def hashObject(
path: Path): TaskR[FileSystem, Map[HashType, MD5Hash]] =
for {
md5 <- MD5HashGenerator.md5File(path)
} yield Map(MD5 -> md5)
override def hashObjectChunk(path: Path,
chunkNumber: Long,
chunkSize: Long)
: TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
for {
md5 <- MD5HashGenerator.md5FileChunk(path,
chunkNumber * chunkSize,
chunkSize)
} yield Map(MD5 -> md5)
override def hex(in: Array[Byte]): TaskR[Hasher, String] =
ZIO(MD5HashGenerator.hex(in))
override def digest(in: String): TaskR[Hasher, Array[Byte]] =
ZIO(MD5HashGenerator.digest(in))
}
}
object Live extends Live
trait Test extends Hasher {
val hashes: AtomicReference[Map[Path, Map[HashType, MD5Hash]]] =
new AtomicReference(Map.empty)
val hashChunks
: AtomicReference[Map[Path, Map[Long, Map[HashType, MD5Hash]]]] =
new AtomicReference(Map.empty)
val hasher: Service = new Service {
override def hashObject(
path: Path): TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
ZIO(hashes.get()(path))
override def hashObjectChunk(path: Path,
chunkNumber: Long,
chunkSize: Long)
: TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
ZIO(hashChunks.get()(path)(chunkNumber))
override def hex(in: Array[Byte]): TaskR[Hasher, String] =
ZIO(MD5HashGenerator.hex(in))
override def digest(in: String): TaskR[Hasher, Array[Byte]] =
ZIO(MD5HashGenerator.digest(in))
}
}
object Test extends Test
final def hashObject(
path: Path): TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
ZIO.accessM(_.hasher hashObject path)
final def hashObjectChunk(
path: Path,
chunkNumber: Long,
chunkSize: Long): TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
ZIO.accessM(_.hasher hashObjectChunk (path, chunkNumber, chunkSize))
final def hex(in: Array[Byte]): TaskR[Hasher, String] =
ZIO.accessM(_.hasher hex in)
final def digest(in: String): TaskR[Hasher, Array[Byte]] =
ZIO.accessM(_.hasher digest in)
}

View file

@ -1,4 +1,4 @@
package net.kemitix.thorp.core package net.kemitix.thorp.core.hasher
import java.io.{File, FileInputStream} import java.io.{File, FileInputStream}
import java.nio.file.Path import java.nio.file.Path
@ -10,7 +10,7 @@ import zio.{Task, TaskR}
import scala.collection.immutable.NumericRange import scala.collection.immutable.NumericRange
object MD5HashGenerator { private object MD5HashGenerator {
val maxBufferSize = 8048 val maxBufferSize = 8048
val defaultBuffer = new Array[Byte](maxBufferSize) val defaultBuffer = new Array[Byte](maxBufferSize)
@ -48,13 +48,11 @@ object MD5HashGenerator {
offset: Long, offset: Long,
endOffset: Long endOffset: Long
) = ) =
FileSystem FileSystem.open(file, offset) >>= { managedFileInputStream =>
.open(file, offset) managedFileInputStream.use { fileInputStream =>
.flatMap { managedFileInputStream => digestFile(fileInputStream, offset, endOffset)
managedFileInputStream.use { fileInputStream =>
digestFile(fileInputStream, offset, endOffset)
}
} }
}
private def digestFile( private def digestFile(
fis: FileInputStream, fis: FileInputStream,

View file

@ -1,14 +0,0 @@
package net.kemitix.thorp.core
import java.nio.file.Path
import net.kemitix.thorp.domain.{HashType, MD5Hash}
import zio.Task
case class DummyHashService(hashes: Map[Path, Map[HashType, MD5Hash]])
extends HashService {
override def hashLocalObject(path: Path): Task[Map[HashType, MD5Hash]] =
Task(hashes(path))
}

View file

@ -10,6 +10,7 @@ import net.kemitix.thorp.config.{
Resource Resource
} }
import net.kemitix.thorp.console._ import net.kemitix.thorp.console._
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.domain.HashType.MD5 import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.domain._ import net.kemitix.thorp.domain._
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
@ -21,30 +22,17 @@ class LocalFileStreamSuite extends FunSpec {
private val source = Resource(this, "upload") private val source = Resource(this, "upload")
private val sourcePath = source.toPath private val sourcePath = source.toPath
private val hashService: HashService = DummyHashService(
Map(
file("root-file") -> Map(MD5 -> MD5HashData.Root.hash),
file("subdir/leaf-file") -> Map(MD5 -> MD5HashData.Leaf.hash)
))
private def file(filename: String) = private def file(filename: String) =
sourcePath.resolve(Paths.get(filename)) sourcePath.resolve(Paths.get(filename))
private val configOptions = ConfigOptions(
List(
ConfigOption.IgnoreGlobalOptions,
ConfigOption.IgnoreUserOptions,
ConfigOption.Source(sourcePath),
ConfigOption.Bucket("aBucket")
))
describe("findFiles") { describe("findFiles") {
it("should find all files") { it("should find all files") {
val expected = Right(Set("subdir/leaf-file", "root-file")) val expected = Right(Set("subdir/leaf-file", "root-file"))
val result = val result =
invoke() invoke()
.map(_.localFiles) .map(_.localFiles)
.map(localFiles => localFiles.map(_.relative.toString)) .map(_.map(_.relative.toString))
.map(_.toSet) .map(_.toSet)
assertResult(expected)(result) assertResult(expected)(result)
} }
@ -61,9 +49,13 @@ class LocalFileStreamSuite extends FunSpec {
} }
private def invoke() = { private def invoke() = {
type TestEnv = Storage with Console with Config with FileSystem type TestEnv = Storage
with Console
with Config
with FileSystem
with Hasher.Test
val testEnv: TestEnv = new Storage.Test with Console.Test with Config.Live val testEnv: TestEnv = new Storage.Test with Console.Test with Config.Live
with FileSystem.Live { with FileSystem.Live with Hasher.Test {
override def listResult: Task[S3ObjectsData] = override def listResult: Task[S3ObjectsData] =
Task.die(new NotImplementedError) Task.die(new NotImplementedError)
override def uploadResult: UIO[StorageQueueEvent] = override def uploadResult: UIO[StorageQueueEvent] =
@ -75,12 +67,23 @@ class LocalFileStreamSuite extends FunSpec {
override def shutdownResult: UIO[StorageQueueEvent] = override def shutdownResult: UIO[StorageQueueEvent] =
Task.die(new NotImplementedError) Task.die(new NotImplementedError)
} }
testEnv.hashes.set(
Map(
file("root-file") -> Map(MD5 -> MD5HashData.Root.hash),
file("subdir/leaf-file") -> Map(MD5 -> MD5HashData.Leaf.hash)
))
val configOptions = ConfigOptions(
List(
ConfigOption.IgnoreGlobalOptions,
ConfigOption.IgnoreUserOptions,
ConfigOption.Source(sourcePath),
ConfigOption.Bucket("aBucket")
))
def testProgram = def testProgram =
for { for {
config <- ConfigurationBuilder.buildConfig(configOptions) config <- ConfigurationBuilder.buildConfig(configOptions)
_ <- Config.set(config) _ <- Config.set(config)
files <- LocalFileStream.findFiles(hashService)(sourcePath) files <- LocalFileStream.findFiles(sourcePath)
} yield files } yield files
new DefaultRuntime {}.unsafeRunSync { new DefaultRuntime {}.unsafeRunSync {

View file

@ -1,83 +0,0 @@
package net.kemitix.thorp.core
import java.nio.file.Paths
import net.kemitix.thorp.config.{ConfigOption, ConfigOptions, ParseConfigLines}
import org.scalatest.FunSpec
class ParseConfigLinesTest extends FunSpec {
describe("parse single lines") {
describe("source") {
it("should parse") {
val expected =
ConfigOptions(List(ConfigOption.Source(Paths.get("/path/to/source"))))
val result =
ParseConfigLines.parseLines(List("source = /path/to/source"))
assertResult(expected)(result)
}
}
describe("bucket") {
it("should parse") {
val expected = ConfigOptions(List(ConfigOption.Bucket("bucket-name")))
val result = ParseConfigLines.parseLines(List("bucket = bucket-name"))
assertResult(expected)(result)
}
}
describe("prefix") {
it("should parse") {
val expected =
ConfigOptions(List(ConfigOption.Prefix("prefix/to/files")))
val result =
ParseConfigLines.parseLines(List("prefix = prefix/to/files"))
assertResult(expected)(result)
}
}
describe("include") {
it("should parse") {
val expected =
ConfigOptions(List(ConfigOption.Include("path/to/include")))
val result =
ParseConfigLines.parseLines(List("include = path/to/include"))
assertResult(expected)(result)
}
}
describe("exclude") {
it("should parse") {
val expected =
ConfigOptions(List(ConfigOption.Exclude("path/to/exclude")))
val result =
ParseConfigLines.parseLines(List("exclude = path/to/exclude"))
assertResult(expected)(result)
}
}
describe("debug - true") {
it("should parse") {
val expected = ConfigOptions(List(ConfigOption.Debug()))
val result = ParseConfigLines.parseLines(List("debug = true"))
assertResult(expected)(result)
}
}
describe("debug - false") {
it("should parse") {
val expected = ConfigOptions()
val result = ParseConfigLines.parseLines(List("debug = false"))
assertResult(expected)(result)
}
}
describe("comment line") {
it("should be ignored") {
val expected = ConfigOptions()
val result = ParseConfigLines.parseLines(List("# ignore me"))
assertResult(expected)(result)
}
}
describe("unrecognised option") {
it("should be ignored") {
val expected = ConfigOptions()
val result = ParseConfigLines.parseLines(List("unsupported = option"))
assertResult(expected)(result)
}
}
}
}

View file

@ -11,6 +11,7 @@ import net.kemitix.thorp.config.{
} }
import net.kemitix.thorp.console._ import net.kemitix.thorp.console._
import net.kemitix.thorp.core.Action.{DoNothing, ToCopy, ToDelete, ToUpload} import net.kemitix.thorp.core.Action.{DoNothing, ToCopy, ToDelete, ToUpload}
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.domain.HashType.MD5 import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.domain._ import net.kemitix.thorp.domain._
import net.kemitix.thorp.filesystem._ import net.kemitix.thorp.filesystem._
@ -25,8 +26,6 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
"create a plan" - { "create a plan" - {
val hashService = SimpleHashService()
"one source" - { "one source" - {
val options: Path => ConfigOptions = val options: Path => ConfigOptions =
source => source =>
@ -46,8 +45,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
val expected = val expected =
Right(List(toUpload(remoteKey, hash, source, file))) Right(List(toUpload(remoteKey, hash, source, file)))
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(emptyS3ObjectData), UIO.succeed(emptyS3ObjectData),
UIO.succeed(Map(file.toPath -> file))) UIO.succeed(Map(file.toPath -> file)))
assertResult(expected)(result) assertResult(expected)(result)
@ -70,8 +68,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
byKey = Map(anOtherKey -> HashModified(aHash, lastModified)) byKey = Map(anOtherKey -> HashModified(aHash, lastModified))
) )
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(s3ObjectsData), UIO.succeed(s3ObjectsData),
UIO.succeed(Map(aFile.toPath -> aFile, UIO.succeed(Map(aFile.toPath -> aFile,
anOtherFile.toPath -> anOtherFile))) anOtherFile.toPath -> anOtherFile)))
@ -94,8 +91,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
byKey = Map(remoteKey -> HashModified(hash, lastModified)) byKey = Map(remoteKey -> HashModified(hash, lastModified))
) )
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(s3ObjectsData), UIO.succeed(s3ObjectsData),
UIO.succeed(Map(file.toPath -> file))) UIO.succeed(Map(file.toPath -> file)))
assertResult(expected)(result) assertResult(expected)(result)
@ -118,8 +114,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
Map(remoteKey -> HashModified(originalHash, lastModified)) Map(remoteKey -> HashModified(originalHash, lastModified))
) )
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(s3ObjectsData), UIO.succeed(s3ObjectsData),
UIO.succeed(Map(file.toPath -> file))) UIO.succeed(Map(file.toPath -> file)))
assertResult(expected)(result) assertResult(expected)(result)
@ -139,8 +134,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
byKey = Map() byKey = Map()
) )
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(s3ObjectsData), UIO.succeed(s3ObjectsData),
UIO.succeed(Map(file.toPath -> file))) UIO.succeed(Map(file.toPath -> file)))
assertResult(expected)(result) assertResult(expected)(result)
@ -165,8 +159,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
byKey = Map(remoteKey -> HashModified(hash, lastModified)) byKey = Map(remoteKey -> HashModified(hash, lastModified))
) )
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(s3ObjectsData), UIO.succeed(s3ObjectsData),
UIO.succeed(Map(file.toPath -> file))) UIO.succeed(Map(file.toPath -> file)))
assertResult(expected)(result) assertResult(expected)(result)
@ -183,8 +176,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
byKey = Map(remoteKey -> HashModified(hash, lastModified)) byKey = Map(remoteKey -> HashModified(hash, lastModified))
) )
val result = val result =
invoke(hashService, invoke(options(source),
options(source),
UIO.succeed(s3ObjectsData), UIO.succeed(s3ObjectsData),
UIO.succeed(Map.empty)) UIO.succeed(Map.empty))
assertResult(expected)(result) assertResult(expected)(result)
@ -222,7 +214,6 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
)) ))
val result = val result =
invoke( invoke(
hashService,
options(firstSource)(secondSource), options(firstSource)(secondSource),
UIO.succeed(emptyS3ObjectData), UIO.succeed(emptyS3ObjectData),
UIO.succeed( UIO.succeed(
@ -248,7 +239,6 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
toUpload(remoteKey1, hash1, firstSource, fileInFirstSource))) toUpload(remoteKey1, hash1, firstSource, fileInFirstSource)))
val result = val result =
invoke( invoke(
hashService,
options(firstSource)(secondSource), options(firstSource)(secondSource),
UIO.succeed(emptyS3ObjectData), UIO.succeed(emptyS3ObjectData),
UIO.succeed( UIO.succeed(
@ -273,8 +263,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
Map(hash2 -> Set(KeyModified(remoteKey2, lastModified))), Map(hash2 -> Set(KeyModified(remoteKey2, lastModified))),
byKey = Map(remoteKey2 -> HashModified(hash2, lastModified))) byKey = Map(remoteKey2 -> HashModified(hash2, lastModified)))
val result = val result =
invoke(hashService, invoke(options(firstSource)(secondSource),
options(firstSource)(secondSource),
UIO.succeed(s3ObjectData), UIO.succeed(s3ObjectData),
UIO.succeed( UIO.succeed(
Map(fileInSecondSource.toPath -> fileInSecondSource))) Map(fileInSecondSource.toPath -> fileInSecondSource)))
@ -296,8 +285,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
Map(hash1 -> Set(KeyModified(remoteKey1, lastModified))), Map(hash1 -> Set(KeyModified(remoteKey1, lastModified))),
byKey = Map(remoteKey1 -> HashModified(hash1, lastModified))) byKey = Map(remoteKey1 -> HashModified(hash1, lastModified)))
val result = val result =
invoke(hashService, invoke(options(firstSource)(secondSource),
options(firstSource)(secondSource),
UIO.succeed(s3ObjectData), UIO.succeed(s3ObjectData),
UIO.succeed( UIO.succeed(
Map(fileInFirstSource.toPath -> fileInFirstSource))) Map(fileInFirstSource.toPath -> fileInFirstSource)))
@ -314,8 +302,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
val s3ObjectData = S3ObjectsData(byKey = val s3ObjectData = S3ObjectsData(byKey =
Map(remoteKey1 -> HashModified(MD5Hash(""), lastModified))) Map(remoteKey1 -> HashModified(MD5Hash(""), lastModified)))
val result = val result =
invoke(hashService, invoke(options(firstSource)(secondSource),
options(firstSource)(secondSource),
UIO.succeed(s3ObjectData), UIO.succeed(s3ObjectData),
UIO.succeed(Map.empty)) UIO.succeed(Map.empty))
assertResult(expected)(result) assertResult(expected)(result)
@ -326,12 +313,13 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
} }
def md5Hash(file: File): MD5Hash = { def md5Hash(file: File): MD5Hash = {
object TestEnv extends Hasher.Live with FileSystem.Live
new DefaultRuntime {} new DefaultRuntime {}
.unsafeRunSync { .unsafeRunSync {
hashService Hasher
.hashLocalObject(file.toPath) .hashObject(file.toPath)
.map(_.get(MD5)) .map(_.get(MD5))
.provide(FileSystem.Live) .provide(TestEnv)
} }
.toEither .toEither
.toOption .toOption
@ -361,14 +349,13 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
ConfigOptions(List(configOptions: _*)) ConfigOptions(List(configOptions: _*))
private def invoke( private def invoke(
hashService: HashService,
configOptions: ConfigOptions, configOptions: ConfigOptions,
result: Task[S3ObjectsData], result: Task[S3ObjectsData],
files: Task[Map[Path, File]] files: Task[Map[Path, File]]
) = { ) = {
type TestEnv = Storage with Console with Config with FileSystem type TestEnv = Storage with Console with Config with FileSystem with Hasher
val testEnv: TestEnv = new Storage.Test with Console.Test with Config.Live val testEnv: TestEnv = new Storage.Test with Console.Test with Config.Live
with FileSystem.Live { with FileSystem.Live with Hasher.Live {
override def listResult: Task[S3ObjectsData] = result override def listResult: Task[S3ObjectsData] = result
override def uploadResult: UIO[StorageQueueEvent] = override def uploadResult: UIO[StorageQueueEvent] =
Task.die(new NotImplementedError) Task.die(new NotImplementedError)
@ -384,7 +371,7 @@ class PlanBuilderTest extends FreeSpec with TemporaryFolder {
for { for {
config <- ConfigurationBuilder.buildConfig(configOptions) config <- ConfigurationBuilder.buildConfig(configOptions)
_ <- Config.set(config) _ <- Config.set(config)
plan <- PlanBuilder.createPlan(hashService) plan <- PlanBuilder.createPlan
} yield plan } yield plan
new DefaultRuntime {} new DefaultRuntime {}

View file

@ -1,4 +1,4 @@
package net.kemitix.thorp.core package net.kemitix.thorp.core.hasher
import java.nio.file.Path import java.nio.file.Path
@ -12,7 +12,7 @@ class MD5HashGeneratorTest extends FunSpec {
describe("md5File()") { describe("md5File()") {
describe("read a small file (smaller than buffer)") { describe("read a small file (smaller than buffer)") {
val path = Resource(this, "upload/root-file").toPath val path = Resource(this, "../upload/root-file").toPath
it("should generate the correct hash") { it("should generate the correct hash") {
val expected = Right(Root.hash) val expected = Right(Root.hash)
val result = invoke(path) val result = invoke(path)
@ -21,7 +21,7 @@ class MD5HashGeneratorTest extends FunSpec {
} }
describe("read a large file (bigger than buffer)") { describe("read a large file (bigger than buffer)") {
val path = Resource(this, "big-file").toPath val path = Resource(this, "../big-file").toPath
it("should generate the correct hash") { it("should generate the correct hash") {
val expected = Right(BigFile.hash) val expected = Right(BigFile.hash)
val result = invoke(path) val result = invoke(path)
@ -39,7 +39,7 @@ class MD5HashGeneratorTest extends FunSpec {
describe("md5FileChunk") { describe("md5FileChunk") {
describe("read chunks of file") { describe("read chunks of file") {
val path = Resource(this, "big-file").toPath val path = Resource(this, "../big-file").toPath
it("should generate the correct hash for first chunk of the file") { it("should generate the correct hash for first chunk of the file") {
val part1 = BigFile.Part1 val part1 = BigFile.Part1
val expected = Right(part1.hash.hash) val expected = Right(part1.hash.hash)

View file

@ -7,7 +7,7 @@ sealed trait Filter
object Filter { object Filter {
def isIncluded(filters: List[Filter])(p: Path): Boolean = { def isIncluded(p: Path)(filters: List[Filter]): Boolean = {
sealed trait State sealed trait State
case class Unknown() extends State case class Unknown() extends State
case class Accepted() extends State case class Accepted() extends State

View file

@ -0,0 +1,6 @@
package net.kemitix.thorp.domain
trait Monoid[T] {
def zero: T
def op(t1: T, t2: T): T
}

View file

@ -15,12 +15,9 @@ import java.nio.file.Path
case class Sources( case class Sources(
paths: List[Path] paths: List[Path]
) { ) {
def ++(path: Path): Sources = this ++ List(path) def +(path: Path)(implicit m: Monoid[Sources]): Sources = this ++ List(path)
def ++(otherPaths: List[Path]): Sources = def ++(otherPaths: List[Path])(implicit m: Monoid[Sources]): Sources =
Sources(otherPaths.foldLeft(paths) { (acc, path) => m.op(this, Sources(otherPaths))
if (acc.contains(path)) acc
else acc ++ List(path)
})
/** /**
* Returns the source path for the given path. * Returns the source path for the given path.
@ -28,3 +25,17 @@ case class Sources(
def forPath(path: Path): Path = def forPath(path: Path): Path =
paths.find(source => path.startsWith(source)).get paths.find(source => path.startsWith(source)).get
} }
object Sources {
final val emptySources = Sources(List.empty)
implicit def sourcesAppendMonoid: Monoid[Sources] = new Monoid[Sources] {
override def zero: Sources = emptySources
override def op(t1: Sources, t2: Sources): Sources =
Sources(t2.paths.foldLeft(t1.paths) { (acc, path) =>
if (acc.contains(path)) acc
else acc ++ List(path)
})
}
}

View file

@ -86,7 +86,7 @@ class FiltersSuite extends FunSpec {
} }
describe("isIncluded") { describe("isIncluded") {
def invoke(filters: List[Filter]) = { def invoke(filters: List[Filter]) = {
paths.filter(path => Filter.isIncluded(filters)(path)) paths.filter(path => Filter.isIncluded(path)(filters))
} }
describe("when there are no filters") { describe("when there are no filters") {

View file

@ -1,4 +1,4 @@
package net.kemitix.thorp.core package net.kemitix.thorp.domain
import java.io.{File, IOException, PrintWriter} import java.io.{File, IOException, PrintWriter}
import java.nio.file.attribute.BasicFileAttributes import java.nio.file.attribute.BasicFileAttributes

View file

@ -18,7 +18,7 @@ object FileSystem {
def fileExists(file: File): ZIO[FileSystem, Throwable, Boolean] def fileExists(file: File): ZIO[FileSystem, Throwable, Boolean]
def openManagedFileInputStream(file: File, offset: Long = 0L) def openManagedFileInputStream(file: File, offset: Long = 0L)
: TaskR[FileSystem, ZManaged[Any, Throwable, FileInputStream]] : TaskR[FileSystem, ZManaged[Any, Throwable, FileInputStream]]
def fileLines(file: File): TaskR[FileSystem, List[String]] def fileLines(file: File): TaskR[FileSystem, Seq[String]]
} }
trait Live extends FileSystem { trait Live extends FileSystem {
override val filesystem: Service = new Service { override val filesystem: Service = new Service {
@ -42,17 +42,11 @@ object FileSystem {
ZIO(ZManaged.make(acquire)(release)) ZIO(ZManaged.make(acquire)(release))
} }
override def fileLines(file: File): TaskR[FileSystem, List[String]] = { override def fileLines(file: File): TaskR[FileSystem, Seq[String]] = {
def acquire = ZIO(Files.lines(file.toPath)) def acquire = ZIO(Files.lines(file.toPath))
def release(lines: stream.Stream[String]) =
ZIO.effectTotal(lines.close())
def use(lines: stream.Stream[String]) = def use(lines: stream.Stream[String]) =
ZIO.effectTotal(lines.iterator.asScala.toList) ZIO.effectTotal(lines.iterator.asScala.toList)
acquire.bracketAuto(use)
ZIO.bracket(acquire)(release)(use)
} }
} }
} }
@ -84,6 +78,6 @@ object FileSystem {
: TaskR[FileSystem, ZManaged[FileSystem, Throwable, FileInputStream]] = : TaskR[FileSystem, ZManaged[FileSystem, Throwable, FileInputStream]] =
ZIO.accessM(_.filesystem openManagedFileInputStream (file, offset)) ZIO.accessM(_.filesystem openManagedFileInputStream (file, offset))
final def lines(file: File): TaskR[FileSystem, List[String]] = final def lines(file: File): TaskR[FileSystem, Seq[String]] =
ZIO.accessM(_.filesystem fileLines (file)) ZIO.accessM(_.filesystem fileLines (file))
} }

View file

@ -14,48 +14,34 @@ import zio.{IO, Task, UIO}
trait Copier { trait Copier {
def copy(amazonS3: AmazonS3.Client)( def copy(amazonS3: AmazonS3.Client)(
request: Request): UIO[StorageQueueEvent] =
copyObject(amazonS3)(request)
.fold(foldFailure(request.sourceKey, request.targetKey),
foldSuccess(request.sourceKey, request.targetKey))
case class Request(
bucket: Bucket, bucket: Bucket,
sourceKey: RemoteKey, sourceKey: RemoteKey,
hash: MD5Hash, hash: MD5Hash,
targetKey: RemoteKey targetKey: RemoteKey
): UIO[StorageQueueEvent] = )
copyObject(amazonS3)(bucket, sourceKey, hash, targetKey)
.fold(foldFailure(sourceKey, targetKey),
foldSuccess(sourceKey, targetKey))
private def copyObject(amazonS3: AmazonS3.Client)( private def copyObject(amazonS3: AmazonS3.Client)(request: Request) =
bucket: Bucket,
sourceKey: RemoteKey,
hash: MD5Hash,
targetKey: RemoteKey
): IO[S3ClientException, CopyObjectResult] = {
def handleResult
: Option[CopyObjectResult] => IO[S3ClientException, CopyObjectResult] =
maybeResult =>
IO.fromEither {
maybeResult
.toRight(HashError)
}
def handleError: Throwable => IO[S3ClientException, CopyObjectResult] =
error =>
Task.fail {
CopyError(error)
}
val request =
new CopyObjectRequest(
bucket.name,
sourceKey.key,
bucket.name,
targetKey.key
).withMatchingETagConstraint(hash.hash)
amazonS3 amazonS3
.copyObject(request) .copyObject(copyObjectRequest(request))
.fold(handleError, handleResult) .fold(
error => Task.fail(CopyError(error)),
result => IO.fromEither(result.toRight(HashError))
)
.flatten .flatten
}
private def copyObjectRequest(copyRequest: Request) =
new CopyObjectRequest(
copyRequest.bucket.name,
copyRequest.sourceKey.key,
copyRequest.bucket.name,
copyRequest.targetKey.key
).withMatchingETagConstraint(copyRequest.hash.hash)
private def foldFailure( private def foldFailure(
sourceKey: RemoteKey, sourceKey: RemoteKey,

View file

@ -32,11 +32,7 @@ trait Lister {
token => request.withContinuationToken(token) token => request.withContinuationToken(token)
def fetchBatch: ListObjectsV2Request => TaskR[Console, Batch] = def fetchBatch: ListObjectsV2Request => TaskR[Console, Batch] =
request => request => ListerLogger.logFetchBatch *> tryFetchBatch(amazonS3)(request)
for {
_ <- ListerLogger.logFetchBatch
batch <- tryFetchBatch(amazonS3)(request)
} yield batch
def fetchMore: Option[Token] => TaskR[Console, Stream[S3ObjectSummary]] = { def fetchMore: Option[Token] => TaskR[Console, Stream[S3ObjectSummary]] = {
case None => TaskR.succeed(Stream.empty) case None => TaskR.succeed(Stream.empty)

View file

@ -1,35 +0,0 @@
package net.kemitix.thorp.storage.aws
import java.nio.file.Path
import net.kemitix.thorp.core.{HashService, MD5HashGenerator}
import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.domain.{HashType, MD5Hash}
import net.kemitix.thorp.filesystem.FileSystem
import zio.TaskR
trait S3HashService extends HashService {
/**
* Generates an MD5 Hash and an multi-part ETag
*
* @param path the local path to scan
* @return a set of hash values
*/
override def hashLocalObject(
path: Path
): TaskR[FileSystem, Map[HashType, MD5Hash]] =
for {
md5 <- MD5HashGenerator.md5File(path)
etag <- ETagGenerator.eTag(path).map(MD5Hash(_))
} yield
Map(
MD5 -> md5,
ETag -> etag
)
}
object S3HashService extends S3HashService {
lazy val defaultHashService: HashService = S3HashService
}

View file

@ -38,7 +38,7 @@ object S3Storage {
sourceKey: RemoteKey, sourceKey: RemoteKey,
hash: MD5Hash, hash: MD5Hash,
targetKey: RemoteKey): UIO[StorageQueueEvent] = targetKey: RemoteKey): UIO[StorageQueueEvent] =
Copier.copy(client)(bucket, sourceKey, hash, targetKey) Copier.copy(client)(Copier.Request(bucket, sourceKey, hash, targetKey))
override def delete(bucket: Bucket, override def delete(bucket: Bucket,
remoteKey: RemoteKey): UIO[StorageQueueEvent] = remoteKey: RemoteKey): UIO[StorageQueueEvent] =

View file

@ -28,22 +28,17 @@ trait Uploader {
.catchAll(handleError(localFile.remoteKey)) .catchAll(handleError(localFile.remoteKey))
private def handleError(remoteKey: RemoteKey)(e: Throwable) = private def handleError(remoteKey: RemoteKey)(e: Throwable) =
UIO.succeed(ErrorQueueEvent(Action.Upload(remoteKey.key), remoteKey, e)) UIO(ErrorQueueEvent(Action.Upload(remoteKey.key), remoteKey, e))
private def transfer(transferManager: => AmazonTransferManager)( private def transfer(transferManager: => AmazonTransferManager)(
localFile: LocalFile, localFile: LocalFile,
bucket: Bucket, bucket: Bucket,
uploadEventListener: UploadEventListener uploadEventListener: UploadEventListener
) = { ) =
val listener = progressListener(uploadEventListener) request(localFile, bucket, progressListener(uploadEventListener)) >>=
for { dispatch(transferManager)
putObjectRequest <- request(localFile, bucket, listener)
event <- dispatch(transferManager, putObjectRequest)
} yield event
}
private def dispatch( private def dispatch(transferManager: AmazonTransferManager)(
transferManager: AmazonTransferManager,
putObjectRequest: PutObjectRequest putObjectRequest: PutObjectRequest
) = { ) = {
transferManager transferManager

View file

@ -1,29 +1,31 @@
package net.kemitix.thorp.storage.aws package net.kemitix.thorp.storage.aws.hasher
import java.nio.file.Path import java.nio.file.Path
import com.amazonaws.services.s3.model.PutObjectRequest import com.amazonaws.services.s3.model.PutObjectRequest
import com.amazonaws.services.s3.transfer.TransferManagerConfiguration import com.amazonaws.services.s3.transfer.TransferManagerConfiguration
import com.amazonaws.services.s3.transfer.internal.TransferManagerUtils import com.amazonaws.services.s3.transfer.internal.TransferManagerUtils
import net.kemitix.thorp.core.MD5HashGenerator import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.domain.MD5Hash import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import zio.{TaskR, ZIO} import zio.{TaskR, ZIO}
trait ETagGenerator { private trait ETagGenerator {
def eTag( def eTag(
path: Path path: Path
): TaskR[FileSystem, String] = { ): TaskR[Hasher with FileSystem, String] = {
val partSize = calculatePartSize(path) val partSize = calculatePartSize(path)
val parts = numParts(path.toFile.length, partSize) val parts = numParts(path.toFile.length, partSize)
ZIO eTagHex(path, partSize, parts)
.foreach(partsIndex(parts))(digestChunk(path, partSize))
.map(concatenateDigests)
.map(MD5HashGenerator.hex)
.map(hash => s"$hash-$parts") .map(hash => s"$hash-$parts")
} }
private def eTagHex(path: Path, partSize: Long, parts: Long) =
ZIO
.foreach(partsIndex(parts))(digestChunk(path, partSize))
.map(concatenateDigests) >>= Hasher.hex
private def partsIndex(parts: Long) = private def partsIndex(parts: Long) =
Range.Long(0, parts, 1).toList Range.Long(0, parts, 1).toList
@ -51,14 +53,10 @@ trait ETagGenerator {
path: Path, path: Path,
chunkSize: Long chunkSize: Long
)(chunkNumber: Long) = )(chunkNumber: Long) =
hashChunk(path, chunkNumber, chunkSize).map(_.digest) Hasher
.hashObjectChunk(path, chunkNumber, chunkSize)
def hashChunk( .map(_(MD5))
path: Path, .map(_.digest)
chunkNumber: Long,
chunkSize: Long
): TaskR[FileSystem, MD5Hash] =
MD5HashGenerator.md5FileChunk(path, chunkNumber * chunkSize, chunkSize)
def offsets( def offsets(
totalFileSizeBytes: Long, totalFileSizeBytes: Long,
@ -67,4 +65,4 @@ trait ETagGenerator {
Range.Long(0, totalFileSizeBytes, optimalPartSize).toList Range.Long(0, totalFileSizeBytes, optimalPartSize).toList
} }
object ETagGenerator extends ETagGenerator private object ETagGenerator extends ETagGenerator

View file

@ -0,0 +1,45 @@
package net.kemitix.thorp.storage.aws.hasher
import java.nio.file.Path
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.core.hasher.Hasher.Live.{hasher => CoreHasher}
import net.kemitix.thorp.core.hasher.Hasher.Service
import net.kemitix.thorp.domain.{HashType, MD5Hash}
import net.kemitix.thorp.filesystem.FileSystem
import net.kemitix.thorp.storage.aws.ETag
import zio.TaskR
object S3Hasher {
trait Live extends Hasher {
val hasher: Service = new Service {
/**
* Generates an MD5 Hash and an multi-part ETag
*
* @param path the local path to scan
* @return a set of hash values
*/
override def hashObject(
path: Path): TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
for {
base <- CoreHasher.hashObject(path)
etag <- ETagGenerator.eTag(path).map(MD5Hash(_))
} yield base + (ETag -> etag)
override def hashObjectChunk(path: Path,
chunkNumber: Long,
chunkSize: Long)
: TaskR[Hasher with FileSystem, Map[HashType, MD5Hash]] =
CoreHasher.hashObjectChunk(path, chunkNumber, chunkSize)
override def hex(in: Array[Byte]): TaskR[Hasher, String] =
CoreHasher.hex(in)
override def digest(in: String): TaskR[Hasher, Array[Byte]] =
CoreHasher.digest(in)
}
}
}

View file

@ -42,7 +42,8 @@ trait AmazonS3ClientTestFixture extends MockFactory {
sourceKey: RemoteKey, sourceKey: RemoteKey,
hash: MD5Hash, hash: MD5Hash,
targetKey: RemoteKey): UIO[StorageQueueEvent] = targetKey: RemoteKey): UIO[StorageQueueEvent] =
Copier.copy(client)(bucket, sourceKey, hash, targetKey) Copier.copy(client)(
Copier.Request(bucket, sourceKey, hash, targetKey))
override def delete(bucket: Bucket, override def delete(bucket: Bucket,
remoteKey: RemoteKey): UIO[StorageQueueEvent] = remoteKey: RemoteKey): UIO[StorageQueueEvent] =

View file

@ -88,7 +88,8 @@ class CopierTest extends FreeSpec {
amazonS3Client: AmazonS3.Client amazonS3Client: AmazonS3.Client
) = ) =
runtime.unsafeRunSync { runtime.unsafeRunSync {
Copier.copy(amazonS3Client)(bucket, sourceKey, hash, targetKey) Copier.copy(amazonS3Client)(
Copier.Request(bucket, sourceKey, hash, targetKey))
}.toEither }.toEither
} }

View file

@ -1,18 +1,20 @@
package net.kemitix.thorp.storage.aws package net.kemitix.thorp.storage.aws.hasher
import java.nio.file.Path import java.nio.file.Path
import com.amazonaws.services.s3.transfer.TransferManagerConfiguration import com.amazonaws.services.s3.transfer.TransferManagerConfiguration
import net.kemitix.thorp.config.Resource import net.kemitix.thorp.config.Resource
import net.kemitix.thorp.core.hasher.Hasher
import net.kemitix.thorp.domain.HashType.MD5
import net.kemitix.thorp.filesystem.FileSystem import net.kemitix.thorp.filesystem.FileSystem
import org.scalatest.FunSpec import org.scalatest.FunSpec
import zio.DefaultRuntime import zio.DefaultRuntime
class ETagGeneratorTest extends FunSpec { class ETagGeneratorTest extends FunSpec {
private val runtime = new DefaultRuntime {} object TestEnv extends Hasher.Live with FileSystem.Live
private val bigFile = Resource(this, "big-file") private val bigFile = Resource(this, "../big-file")
private val bigFilePath = bigFile.toPath private val bigFilePath = bigFile.toPath
private val configuration = new TransferManagerConfiguration private val configuration = new TransferManagerConfiguration
private val chunkSize = 1200000 private val chunkSize = 1200000
@ -41,14 +43,16 @@ class ETagGeneratorTest extends FunSpec {
md5Hashes.foreach { md5Hashes.foreach {
case (hash, index) => case (hash, index) =>
assertResult(Right(hash))( assertResult(Right(hash))(
invoke(bigFilePath, index, chunkSize).map(_.hash)) invoke(bigFilePath, index, chunkSize)
.map(_(MD5))
.map(_.hash))
} }
} }
def invoke(path: Path, index: Long, size: Long) = def invoke(path: Path, index: Long, size: Long) =
new DefaultRuntime {}.unsafeRunSync { new DefaultRuntime {}.unsafeRunSync {
ETagGenerator Hasher
.hashChunk(path, index, size) .hashObjectChunk(path, index, size)
.provide(FileSystem.Live) .provide(TestEnv)
}.toEither }.toEither
} }
@ -58,12 +62,13 @@ class ETagGeneratorTest extends FunSpec {
val result = invoke(bigFilePath) val result = invoke(bigFilePath)
assertResult(Right(expected))(result) assertResult(Right(expected))(result)
} }
def invoke(path: Path) = def invoke(path: Path) = {
new DefaultRuntime {}.unsafeRunSync { new DefaultRuntime {}.unsafeRunSync {
ETagGenerator ETagGenerator
.eTag(path) .eTag(path)
.provide(FileSystem.Live) .provide(TestEnv)
}.toEither }.toEither
}
} }
} }