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

Avoid full metadata call during internal topic creation process #337

Merged
merged 3 commits into from
Jun 28, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
4 changes: 3 additions & 1 deletion .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -9,4 +9,6 @@ build
TestResults

.idea/
.vscode/
.vscode/

confidential
38 changes: 28 additions & 10 deletions core/Mock/Sync/SyncAdminClient.cs
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
using Confluent.Kafka.Admin;
using System;
using System.Collections.Generic;
using System.Linq;
using System.Threading.Tasks;
using Streamiz.Kafka.Net.Mock.Kafka;

Expand All @@ -10,11 +11,13 @@ namespace Streamiz.Kafka.Net.Mock.Sync
internal class SyncAdminClient : BasedAdminClient
{
private readonly SyncProducer producer;
private readonly bool _autoCreateTopic;
private AdminClientConfig config;

public SyncAdminClient(SyncProducer producer)
public SyncAdminClient(SyncProducer producer, bool autoCreateTopic)
{
this.producer = producer;
_autoCreateTopic = autoCreateTopic;
}

internal void UseConfig(AdminClientConfig config)
Expand Down Expand Up @@ -66,20 +69,35 @@ public override void Dispose()
public override Metadata GetMetadata(string topic, TimeSpan timeout)
{
var error = new Error(ErrorCode.NoError);

var brokersMetadata = new List<BrokerMetadata> {
new BrokerMetadata(1, "localhost", 9092)
};

var partitionsMetadata = new List<PartitionMetadata>
var topics = producer.GetAllTopics();
var brokersMetadata = new List<BrokerMetadata>
{
new PartitionMetadata(1, 1, new int[1]{1}, new int[1]{1}, error)
new(1, "localhost", 9092)
};

if (topics.Contains(topic))
{
var partitionsMetadata = new List<PartitionMetadata>
{
new(1, 1, new int[1] { 1 }, new int[1] { 1 }, error)
};

var topicMetadata = new TopicMetadata(topic, partitionsMetadata, error);
var topicMetadata = new TopicMetadata(topic, partitionsMetadata, error);

return new Metadata(brokersMetadata,
new List<TopicMetadata>() { topicMetadata },
1, "localhost");
}

if (_autoCreateTopic)
{
// auto create topic if not exist
producer.CreateTopic(topic);
return GetMetadata(topic, timeout);
}

return new Metadata(brokersMetadata,
new List<TopicMetadata>() { topicMetadata },
new List<TopicMetadata>(),
1, "localhost");
}

Expand Down
4 changes: 2 additions & 2 deletions core/Mock/Sync/SyncKafkaSupplier.cs
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,10 @@ internal class SyncKafkaSupplier : IKafkaSupplier
protected SyncProducer producer = null;
protected SyncAdminClient admin = null;

public SyncKafkaSupplier()
public SyncKafkaSupplier(bool autoCreateTopic = true)
{
producer = new SyncProducer();
admin = new SyncAdminClient(producer);
admin = new SyncAdminClient(producer, autoCreateTopic);
}

public virtual IAdminClient GetAdmin(AdminClientConfig config)
Expand Down
2 changes: 1 addition & 1 deletion core/Processors/DefaultTopicManager.cs
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ async Task<IEnumerable<string>> Run()
var defaultConfig = new Dictionary<string, string>();
var topicsNewCreated = new List<string>();
var topicsToCreate = new List<string>();
var metadata = AdminClient.GetMetadata(timeout);

// 1. get source topic partition
// 2. check if changelog exist, :
Expand All @@ -58,6 +57,7 @@ async Task<IEnumerable<string>> Run()
// 3. if changelog doesn't exist, create it with partition number and configuration
foreach (var t in topics)
{
var metadata = AdminClient.GetMetadata(t.Key, timeout);
var numberPartitions = GetNumberPartitionForTopic(metadata, t.Key);
if (numberPartitions == 0)
{
Expand Down
4 changes: 2 additions & 2 deletions core/State/RocksDb/RocksDbOptions.cs
Original file line number Diff line number Diff line change
Expand Up @@ -1191,9 +1191,9 @@ public RocksDbOptions SetOptimizeFiltersForHits(int value)
/// <param name="p3"></param>
/// <param name="p4"></param>
/// <returns>the instance of the current object</returns>
public RocksDbOptions SetPlainTableFactory(uint p1, int p2, double p3, ulong p4)
public RocksDbOptions SetPlainTableFactory(uint user_key_len, int bloom_bits_per_key, double hash_table_ratio, int index_sparseness, int huge_page_tlb_size, char encoding_type, bool full_scan_mode, bool store_index_in_file)
{
columnFamilyOptions.SetPlainTableFactory(p1, p2, p3, p4);
columnFamilyOptions.SetPlainTableFactory(user_key_len, bloom_bits_per_key, hash_table_ratio, index_sparseness, huge_page_tlb_size, encoding_type, full_scan_mode, store_index_in_file);
return this;
}

Expand Down
2 changes: 1 addition & 1 deletion core/Streamiz.Kafka.Net.csproj
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
<PackageReference Include="Microsoft.Extensions.Logging.Abstractions" Version="6.0.1" />
<PackageReference Include="Microsoft.Extensions.Logging.Console" Version="6.0.0" />
<PackageReference Include="Newtonsoft.Json" Version="13.0.1" />
<PackageReference Include="RocksDB" Version="7.7.3.33461" />
<PackageReference Include="RocksDB" Version="8.11.3.46984" />
<PackageReference Include="Confluent.Kafka" Version="2.4.0" />
<PackageReference Include="Microsoft.CSharp" Version="4.7.0" />
<PackageReference Include="System.Dynamic.Runtime" Version="4.3.0" />
Expand Down
6 changes: 2 additions & 4 deletions samples/sample-stream/Program.cs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ public static async Task Main(string[] args)
PartitionAssignmentStrategy = PartitionAssignmentStrategy.RoundRobin,
Partitioner = Partitioner.ConsistentRandom,
Debug = "broker,topic,msg",
ClientId = "ttoot",
Logger = LoggerFactory.Create((b) =>
{
b.AddConsole();
Expand All @@ -50,7 +49,7 @@ private static Topology BuildTopology()
TimeSpan windowSize = TimeSpan.FromHours(1);

var builder = new StreamBuilder();
/* builder.Stream<string, string>("input")
builder.Stream<string, string>("input")
.GroupByKey()
.WindowedBy(TumblingWindowOptions.Of(windowSize))
.Count(RocksDbWindows.As<string, long>("count-store")
Expand All @@ -61,9 +60,8 @@ private static Topology BuildTopology()
.Map((k,v) => new KeyValuePair<string,string>(k.ToString(), v.ToString()))
.To("output",
new StringSerDes(),
new StringSerDes());*/
new StringSerDes());

builder.GlobalTable<string, string>("global");

return builder.Build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ public class DefaultTopicManagerTests
[SetUp]
public void Begin()
{
kafkaSupplier = new SyncKafkaSupplier();
kafkaSupplier = new SyncKafkaSupplier(false);
}

[TearDown]
Expand Down Expand Up @@ -47,7 +47,8 @@ public void ApplyInternalChangelogTopics()
NumberPartitions = 1
});

var r = manager.ApplyAsync(0, topics).GetAwaiter().GetResult().ToList();
var r = manager.ApplyAsync(0, topics)
.GetAwaiter().GetResult().ToList();

Assert.AreEqual(2, r.Count);
Assert.AreEqual("topic", r[0]);
Expand Down
2 changes: 1 addition & 1 deletion test/Streamiz.Kafka.Net.Tests/Public/KafkaStreamTests.cs
Original file line number Diff line number Diff line change
Expand Up @@ -633,7 +633,7 @@ public async Task BuildGlobalStateStore()
config.PollMs = 1;

var builder = new StreamBuilder();
builder.GlobalTable<string, string>("test", InMemory.As<string,string>("store"));
builder.GlobalTable("test", InMemory.As<string,string>("store"));

var supplier = new SyncKafkaSupplier();
var producer = supplier.GetProducer(new ProducerConfig());
Expand Down
8 changes: 1 addition & 7 deletions test/Streamiz.Kafka.Net.Tests/Public/RocksDbOptionsTests.cs
Original file line number Diff line number Diff line change
Expand Up @@ -65,11 +65,8 @@ public void Begin()
.SetArenaBlockSize(arenaBlockSize)
.SetBloomLocality(bloomLocality)
.SetBytesPerSync(bytesPerSync)
// .SetCompactionFilter(IntPtr.Zero)
// .SetCompactionFilterFactory(IntPtr.Zero)
.SetCompactionReadaheadSize(1200)
.SetCompactionStyle(RocksDbSharp.Compaction.Level)
//.SetComparator(IntPtr.Zero)
.SetCompression(RocksDbSharp.Compression.Lz4)
.SetCompressionOptions(1, 2, 3, 4)
.SetCompressionPerLevel(new[] { RocksDbSharp.Compression.Lz4 }, 1)
Expand All @@ -80,9 +77,7 @@ public void Begin()
.SetDeleteObsoleteFilesPeriodMicros(50)
.SetDisableAutoCompactions(1)
.SetEnableWriteThreadAdaptiveYield(true)
//.SetEnv(IntPtr.Zero)
.SetErrorIfExists()
//.SetFifoCompactionOptions(IntPtr.Zero)
.SetErrorIfExists(false)
.SetHardPendingCompactionBytesLimit(1)
.SetHashLinkListRep(12)
.SetHashSkipListRep(56, 4, 2)
Expand Down Expand Up @@ -140,7 +135,6 @@ public void Begin()
.SetPrefixExtractor(SliceTransform.CreateNoOp())
.SetRecycleLogFileNum(1)
.SetReportBgIoStats(true)
.SetPlainTableFactory(1, 23, 4, 2)
.SetMaxBackgroundCompactions(1);
};

Expand Down
Loading