Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDFS: Add sources and flows #965

Merged
merged 35 commits into from
Jun 27, 2018
Merged

HDFS: Add sources and flows #965

merged 35 commits into from
Jun 27, 2018

Conversation

burakkose
Copy link
Contributor

@burakkose burakkose commented May 21, 2018

Ref: #557

In this pull request, you will find the initial work for Hdfs. The PR is work in progress. There are still some TODOs.

  • Documentation (done)
  • Java Tests (done)
  • Support HDFS version 2.x (done)
  • pass-through (done)

While I am working on these, please review the code, suggest new functionalities, and help for testing.

@burakkose burakkose changed the title HDFS/WIP: Add writer connector #557 HDFS/WIP: Add writer connector May 21, 2018
@burakkose burakkose force-pushed the hdfs-writer branch 2 times, most recently from 60fe76e to 3bbd57c Compare May 22, 2018 08:42
Copy link
Member

@ennru ennru left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had a first look. Very good stuff.
How should we think about the HDFS version? Would it work with 2.x?

* Copyright (C) 2016-2018 Lightbend Inc. <http://www.lightbend.com>
*/

package akka.stream.alpakka.hdfs
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be great to move internal stuff into an impl package. That would improve Java Module readiness.

* @param compressionCodec a class encapsulates a streaming compression/decompression pair.
* @param settings Hdfs writing settings
*/
def compressed(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To reduce the API, you could just have HdfsSink.data and let the users connect to Sink.ignore for the other cases.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry I could not understand. HdfsSink.data , HdfsSink.compressed, HdfsSink.sequence are completely different. What exactly do you mean by letting users connect to Sink.ignore

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think users needing these cases might as well use the HdfsFlow and connect it to Sink.ignore themselves.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh, then can we remove Sink implementations?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, just keep the most basic ones. Most users discover they'll need to do something after sending/writing to the destination.
Speaking of that, what we most often need is some kind of pass-through, a value untouched by the flow, but available afterwards (eg. Kafka offsets for committing). Have you thought about that? It would require a wrapper for Writable, AFAICS.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, I was thinking about pass-through like Solr connector. I was just not sure to implement. It will be added soon.

Copy link
Contributor Author

@burakkose burakkose May 23, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would like to consult about it. I was thinking the best strategy for it. Currently, the flow expects ByteString as an input. Whenever it rotates the output, it pushes WriteLog to the downstream.

If we want to implement pass-through, how shall we design it? The dumbest way is

final case class WriteLog[T](path: String, rotation: Int, passThroughs: Seq[T])

However, if we have millions input, keeping this sequence in memory until the flow rotates is super inefficient.

The second idea is

sealed trait OutgoingMessage
final case class RotationMessage(path: String, rotation: Int) extends OutgoingMessage
final case class PassThrough[T](pass: T)  extends OutgoingMessage

So flow can push RotationMessage when it rotates, and for the rest, it will push PassThrough. However, this has a drawback. Let's talk about Kafka example, we write the input to output and send PassThrough message with an offset. If something goes wrong in the flow, and if it does not synchronize the output, we will basically fail, but downstream can already commit this offset.

I actually did not like this idea also. Do you have any idea for it?

Seq(
libraryDependencies ++= Seq(
"org.apache.hadoop" % "hadoop-client" % hadoopVersion, // ApacheV2
"org.typelevel" %% "cats-core" % catsVersion, // MIT,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You might upgrade to Cats 1.1.0.

@ennru ennru added the p:new label May 22, 2018
@burakkose
Copy link
Contributor Author

I am not sure about the HDFS version. How can we provide support for 2.x also? Is there any example for it in Alpakka connectors?

@ennru
Copy link
Member

ennru commented May 22, 2018

No there is no example right now. One way of doing it would be to make the HDFS dependency optional and let the users add their version explicitly.

@burakkose burakkose force-pushed the hdfs-writer branch 2 times, most recently from 3515734 to b687ad5 Compare May 22, 2018 21:35
@burakkose
Copy link
Contributor Author

I have rearranged the project structure with impl package. Shall we move FilePathGenerator, RotationStrategy, SyncStrategy to model.scala.

Can you also check the commit b687ad5 , I have realized that we need to use japi.Pair for Java

@ennru
Copy link
Member

ennru commented May 23, 2018

The build error is an extra comma which is not supported in Scala 2.11 in HdfsWriterSpec.scala:463.

@ennru
Copy link
Member

ennru commented May 23, 2018

We build on Scala 2.11 and 2.12. Since 2.12 you're allowed to have a superfluous comma as in

Seq(
  1,
  2,
)

and that happens to be in there in HdfsWriterSpec.scala:463.

@burakkose
Copy link
Contributor Author

Yes, my late time commit fault :)

@burakkose
Copy link
Contributor Author

I had a comment in the outdated discussion, so I am posting again here as a new discussion.

I would like to consult about it. I was thinking the best strategy for it. Currently, the flow expects ByteString as an input. Whenever it rotates the output, it pushes WriteLog to the downstream.

If we want to implement pass-through, how shall we design it? The dumbest way is

final case class WriteLog[T](path: String, rotation: Int, passThroughs: Seq[T])

However, if we have millions input, keeping this sequence in memory until the flow rotates is super inefficient.

The second idea is

sealed trait OutgoingMessage
final case class RotationMessage(path: String, rotation: Int) extends OutgoingMessage
final case class PassThrough[T](pass: T)  extends OutgoingMessage

So flow can push RotationMessage when it rotates, and for the rest, it will push PassThrough. However, this has a drawback. Let's talk about Kafka example, we write the input to output and send PassThrough message with an offset. If something goes wrong in the flow, and if it does not synchronize the output, we will basically fail, but downstream can already commit this offset.

I actually did not like this idea also. Do you have any idea for it?

@burakkose
Copy link
Contributor Author

Java tests added, and some API simplified for Java usages.

@ennru
Copy link
Member

ennru commented May 29, 2018

Emitting a message for every incoming is the only reasonable way.
The user may accumulate pass-throughs if needed. With Kafka the offsets would be committed on a Rotation/Write message.
An alternative could be to have a type

case class HdfsWritten[T](passThrough: T, status: Option[RotatationMessage])

@burakkose burakkose changed the title HDFS/WIP: Add writer connector HDFS: Add writer connector Jun 2, 2018
@burakkose
Copy link
Contributor Author

@ennru did you have time to have a first look at pass-through.

@burakkose
Copy link
Contributor Author

Here is an update for different versions. I have tested from 2.x to 3.x. Tests passed successfully. Moreover, I have published the library locally, and override the Hadoop version with 2.6 because we use Hadoop 2.6. Data ingestion was smooth. It looks like we do not have any problem with different versions. I added a text in the documentation that mentions the default version and the way to override the default one.

Copy link
Member

@ennru ennru left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks great!
You asked earlier about it: classes that belong to the API may not be hidden in the impl package.
I wonder if making the rotation strategy extendable would be important, some might want to implement a combined time/size rotation strategy.


import scala.concurrent.duration.FiniteDuration

sealed trait RotationStrategy extends Strategy {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At least the RotationStrategy interface should be in the public part of the module, as it is part of the API.
It might make sense to have it extendable.

@burakkose burakkose changed the title HDFS: Add writer connector HDFS: Add sources and flows Jun 9, 2018
@burakkose
Copy link
Contributor Author

I have added Sources and made RotationStrategy and SyncStrategy extendable.

Copy link
Member

@ennru ennru left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm really impressed, this is great work.


### Compressed Data Writer

First, create `CompressionCodec`.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

By adding

"javadoc.org.apache.hadoop.base_url" -> s"https://hadoop.apache.org/docs/r${hadoopVersion}/api/",

in build.sbt you can create links to Hadoop's API via @javadoc.


`FilePathGenerator` provides a functionality to generate rotation path in HDFS.
@scala[@scaladoc[FilePathGenerator](akka.stream.alpakka.hdfs.FilePathGenerator$).]
@java[@scaladoc[FilePathGenerator](akka.stream.alpakka.hdfs.FilePathGenerator$).]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need to use @scala/@java when linking to the same class.

/**
* Internal API
*/
private[hdfs] final class HdfsFlowStage[W, I, C](
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add even @akka.annotation.InternalApi as private[hdfs] doesn't protect from Java users using it.

final case class IncomingMessage[T, C](source: T, passThrough: C)

object IncomingMessage {
// Apply method to use when not using passThrough
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Make these doc comments, please.

HdfsWritingSettings()
}

final case class IncomingMessage[T, C](source: T, passThrough: C)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not super happy with the IncomingMessage name used in several connectors. It can easily become messy when using several connectors in the same codebase. And "incoming" is tied to the point of view of the stage, for the user the data leaves...
Maybe HdfsWriteMessage or HdfsData?

* @param rotationStrategy rotation strategy
* @param settings hdfs writing settings
*/
def dataWithPassThrough[C](
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would P make an easier type parameter name?


sealed abstract class OutgoingMessage[+T]
final case class RotationMessage(path: String, rotation: Int) extends OutgoingMessage[Nothing]
final case class WrittenMessage[T](passThrough: T, inRotation: Int) extends OutgoingMessage[T]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Document the inRotation value.


private[writer] object HdfsWriter {

val NewLineByteArray: Array[Byte] = ByteString(System.getProperty("line.separator")).toArray
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it useful to use the system's separator? It could be provided via the settings instead.


override def preStart(): Unit = {
// Schedule timer to rotate output file
initialRotationStrategy match {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be expressed by something in RotationStrategy instead of the type so it becomes extendible.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you have any hint about this? How can i have a strategy in RotationStrategy for preStart. I would consider to pass stateLogic as a parameter and call schedule there but these methods are not visible. How can i trigger schedule in RotationStrategy.

Copy link
Member

@ennru ennru Jun 18, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You could add an interval: Option[FiniteDuration] to it and use foreach in pre-start to schedule the poll.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did not like the idea of having an optional field in rotation strategy and have a logic for it. I shared the scheduling API with implementation package and use it in new method(preStart) of RotationStrategy. Please check e6135a7 and if you do not like it, I will figure out something else.

* Java API: creates a Flow with [[akka.stream.alpakka.hdfs.impl.HdfsFlowStage]]
* for [[org.apache.hadoop.fs.FSDataOutputStream]]
*
* @param fs file system
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hadoop file system

Copy link
Member

@2m 2m left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking very good! The only larger change I would like to see is to add the IODispatcher attributes where necessary.

private static MiniDFSCluster hdfsCluster = null;
private static ActorSystem system;
private static ActorMaterializer materializer;
private static String destionation = JavaTestUtils.destination();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/destionation/destination

class HdfsReaderSpec extends WordSpecLike with Matchers with BeforeAndAfterAll with BeforeAndAfterEach {

private var hdfsCluster: MiniDFSCluster = _
private val destionation = "/tmp/alpakka/"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/destionation/destination

private static MiniDFSCluster hdfsCluster = null;
private static ActorSystem system;
private static ActorMaterializer materializer;
private static String destionation = JavaTestUtils.destination();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/destionation/destination

class HdfsWriterSpec extends WordSpecLike with Matchers with BeforeAndAfterAll with BeforeAndAfterEach {

private var hdfsCluster: MiniDFSCluster = _
private val destionation = "/tmp/alpakka/"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/destionation/destination

private val out = Outlet[OutgoingMessage[C]](Logging.simpleName(this) + ".out")

override val shape: FlowShape[HdfsWriteMessage[I, C], OutgoingMessage[C]] = FlowShape(in, out)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As the underlying writing abstraction is java.io.OutputStream which is blocking, we have to signal to the Akka Stream materializer that it should materialize HdfsFlowStage to a separate dispatcher. This allows other parts of the stream to continue uninpacted when this stage is going to block a thread during writing operations.

Therefore add the following here:

override def initialAttributes: Attributes =
  super.initialAttributes and ActorAttributes.IODispatcher

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for this comment. While I was running it, i was also profiling the app. I guess this is a nice improvement.

}
.takeWhile(_._1)
.map(_._2)
Source.fromIterator(() => it)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I assume that SequenceFile.Readers next operation is blocking as well. We will have to put this source on a separate dispatcher then as well: .addAttributes(Attributes(ActorAttributes.IODispatcher)).

codec: CompressionCodec,
chunkSize: Int = 8192
): Source[ByteString, Future[IOResult]] =
StreamConverters.fromInputStream(() => codec.createInputStream(fs.open(path)), chunkSize)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A source created by fromInputStream already runs on the IO dispatcher, therefore it is fine here.

@burakkose
Copy link
Contributor Author

@2m, @ennru did you have a chance to review last changes?

Source.fromIterator(() => it)
Source
.fromIterator(() => it)
.addAttributes(Attributes(IODispatcher))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use ActorAttributes.IODispatcher here as well since the DefaultAttributes.IODispatcher is in the impl package.

@2m
Copy link
Member

2m commented Jun 26, 2018

Thank you for the ping. Looking very good. Just one last nitpick an we are good to merge.

@burakkose
Copy link
Contributor Author

Done

Copy link
Member

@2m 2m left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Awesome work @burakkose!

@2m 2m merged commit 224a180 into akka:master Jun 27, 2018
@2m 2m added this to the 0.20 milestone Jun 27, 2018
@2m 2m mentioned this pull request Jun 27, 2018
@2m 2m added p:hdfs and removed p:new labels Jun 27, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants