Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
32 commits
Select commit Hold shift + click to select a range
5a6029a
Scaffolded `PulsarSource` for DeltaStreamer
Aug 5, 2022
e4e1649
Make `PulsarSource` to be properly configured
Aug 11, 2022
c4d6ccb
Implemented proper offset-handling
Aug 11, 2022
e2c9e3a
Fixing `DebeziumSource` to commit offsets to Kafka
Aug 11, 2022
9e590a2
Fixing compilation
Aug 12, 2022
f132f71
Make admin-endpoint URL configurable as well
Aug 12, 2022
7319ec3
Missing license
Aug 12, 2022
3d9823d
Make pulsar-spark-connector dep be provided
Aug 12, 2022
ebf95c6
Make sure "pulsar-client-api" is packaged into "hudi-utilities-bundle"
Aug 12, 2022
f9836a5
Fetch ending offset from Pulsar (batch API doesn't permit using "late…
Aug 12, 2022
0ffb51a
Fetch latest offset instead of using `MessageId.latest` shortcut
Aug 12, 2022
0c0aa09
Ack latest consumed offset
Aug 12, 2022
eed8b9b
Removing "pulsar-client" jars from "hudi-utilities-bundle" since they…
Aug 12, 2022
35eccfa
Fixing subscription to be exclusive to be able to ack "cumulatively"
Aug 12, 2022
ed2fd41
Canonicalize topic's name properly
Aug 12, 2022
7b892b3
Make `PulsarSource` closeable
Aug 12, 2022
16cf07e
Make `SourceFormatAdapter` closeable;
Aug 12, 2022
5565869
Worked around Pulsar's client failure to shutdown properly
Aug 12, 2022
871afb1
Generate unique subscription-id (for isolation);
Aug 12, 2022
a99ac5b
Tidying up
Aug 12, 2022
db7f5a2
Fixing compilation in Scala 2.11
Aug 12, 2022
1bf346e
Fixing subscription to always start at the beginning of the topic
Aug 12, 2022
7ab92ab
Fix pulsar-spark-connector version for Spark 2.x
Aug 12, 2022
7f0658d
Subscribe in `Durable` mode;
Aug 13, 2022
96b561f
Added `ThreadUtils`;
Aug 13, 2022
76f1335
Fixing compilation
Aug 13, 2022
a3ee924
Aligning configs w/ Kafka
Aug 15, 2022
d2e6816
Fixed dep configuration
Aug 15, 2022
f3399b4
Tidying up
Aug 15, 2022
b71d1a0
Reverting to previous Pulsar version
Aug 17, 2022
79ae2b3
Reverting unrelated changes
Aug 17, 2022
513b7d8
Added graceful shutdown timeout
Aug 18, 2022
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,21 @@ package org.apache.hudi

import org.apache.hudi.common.config.TypedProperties

import java.{util => ju}
import scala.collection.JavaConverters

object HoodieConversionUtils {

/**
* Converts Java's [[ju.Map]] into Scala's (immutable) [[Map]] (by default [[JavaConverters]] convert to
* a mutable one)
*/
def mapAsScalaImmutableMap[K, V](map: ju.Map[K, V]): Map[K, V] = {
// NOTE: We have to use deprecated [[JavaConversions]] to stay compatible w/ Scala 2.11
import scala.collection.JavaConversions.mapAsScalaMap
map.toMap
}

def toJavaOption[T](opt: Option[T]): org.apache.hudi.common.util.Option[T] =
if (opt.isDefined) org.apache.hudi.common.util.Option.of(opt.get) else org.apache.hudi.common.util.Option.empty()

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hudi.common.util;

import java.util.Arrays;
import java.util.List;

public class ThreadUtils {

/**
* Fetches all active threads currently running in the JVM
*/
public static List<Thread> collectActiveThreads() {
ThreadGroup threadGroup = Thread.currentThread().getThreadGroup();
while (threadGroup.getParent() != null) {
threadGroup = threadGroup.getParent();
}

Thread[] activeThreads = new Thread[threadGroup.activeCount()];
threadGroup.enumerate(activeThreads);

return Arrays.asList(activeThreads);
}

}
8 changes: 8 additions & 0 deletions hudi-utilities/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,14 @@
</exclusions>
</dependency>

<!-- Pulsar Spark Connector -->
<dependency>
Copy link
Member

Choose a reason for hiding this comment

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

the bundle needs to change too?

Copy link
Contributor

Choose a reason for hiding this comment

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

yes, wondering how did you happen to test this change w/o the bundle.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This will have to be provided. We don't want to include this into our bundle (our bundles are neutral now, so we can't include it in "hudi-utilities-bundle")

<groupId>io.streamnative.connectors</groupId>
<artifactId>pulsar-spark-connector_${scala.binary.version}</artifactId>
<version>${pulsar.spark.version}</version>
<scope>provided</scope>
</dependency>

<!-- Logging -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;

import java.io.Closeable;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
Expand Down Expand Up @@ -124,7 +125,7 @@
/**
* Sync's one batch of data to hoodie table.
*/
public class DeltaSync implements Serializable {
public class DeltaSync implements Serializable, Closeable {

private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(DeltaSync.class);
Expand Down Expand Up @@ -895,11 +896,15 @@ private void registerAvroSchemas(Schema sourceSchema, Schema targetSchema) {
* Close all resources.
*/
public void close() {
if (null != writeClient) {
if (writeClient != null) {
writeClient.close();
writeClient = null;
}

if (formatAdapter != null) {
formatAdapter.close();
}

LOG.info("Shutting down embedded timeline server");
if (embeddedTimelineService.isPresent()) {
embeddedTimelineService.get().stop();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.HoodieSparkUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProvider;
Expand All @@ -38,13 +39,16 @@
import org.apache.spark.sql.Row;
import org.apache.spark.sql.types.StructType;

import java.io.Closeable;
import java.io.IOException;

import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;

/**
* Adapts data-format provided by the source to the data-format required by the client (DeltaStreamer).
*/
public final class SourceFormatAdapter {
public final class SourceFormatAdapter implements Closeable {

private final Source source;

Expand Down Expand Up @@ -123,4 +127,15 @@ public InputBatch<Dataset<Row>> fetchNewDataInRowFormat(Option<String> lastCkptS
public Source getSource() {
return source;
}

@Override
public void close() {
if (source instanceof Closeable) {
try {
((Closeable) source).close();
} catch (IOException e) {
throw new HoodieIOException(String.format("Failed to shutdown the source (%s)", source.getClass().getName()), e);
}
}
}
}
Loading