forked from microsoft/Mobius
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathDStreamSamples.cs
397 lines (322 loc) · 15.4 KB
/
DStreamSamples.cs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
// Copyright (c) Microsoft. All rights reserved.
// Licensed under the MIT license. See LICENSE file in the project root for full license information.
using System;
using System.Collections.Generic;
using System.Configuration;
using System.IO;
using System.Linq;
using System.Text;
using System.Threading.Tasks;
using Microsoft.Spark.CSharp.Core;
using Microsoft.Spark.CSharp.Streaming;
using Microsoft.Spark.CSharp.Samples;
using NUnit.Framework;
namespace Microsoft.Spark.CSharp
{
class DStreamSamples
{
private static int count;
private static bool stopFileServer;
private static void StartFileServer(StreamingContext ssc, string directory, string pattern, int loops = 1)
{
string testDir = Path.Combine(directory, "test");
if (!Directory.Exists(testDir))
Directory.CreateDirectory(testDir);
stopFileServer = false;
string[] files = Directory.GetFiles(directory, pattern);
Task.Run(() =>
{
int loop = 0;
while (!stopFileServer)
{
if (loop++ < loops)
{
DateTime now = DateTime.Now;
foreach (string path in files)
{
string text = File.ReadAllText(path);
File.WriteAllText(testDir + "\\" + now.ToBinary() + "_" + Path.GetFileName(path), text);
}
}
System.Threading.Thread.Sleep(200);
}
ssc.Stop();
});
System.Threading.Thread.Sleep(1);
}
[Sample("experimental")]
internal static void DStreamTextFileSample()
{
count = 0;
string directory = SparkCLRSamples.Configuration.SampleDataLocation;
string checkpointPath = Path.Combine(directory, "checkpoint");
SparkContext sc = SparkCLRSamples.SparkContext;
var b = sc.Broadcast<int>(0);
StreamingContext ssc = StreamingContext.GetOrCreate(checkpointPath,
() =>
{
StreamingContext context = new StreamingContext(sc, 2000L); // batch interval is in milliseconds
context.Checkpoint(checkpointPath);
var lines = context.TextFileStream(Path.Combine(directory, "test"));
lines = context.Union(lines, lines);
var words = lines.FlatMap(l => l.Split(new char[] { ' ' }));
var pairs = words.Map(w => new Tuple<string, int>(w, 1));
// since operations like ReduceByKey, Join and UpdateStateByKey are
// separate dstream transformations defined in CSharpDStream.scala
// an extra CSharpRDD is introduced in between these operations
var wordCounts = pairs.ReduceByKey((x, y) => x + y);
var join = wordCounts.Window(2, 2).Join(wordCounts, 2);
var initialStateRdd = sc.Parallelize(new[] { new Tuple<string, int>("AAA", 88), new Tuple<string, int>("BBB", 88) });
var state = join.UpdateStateByKey((x, y) => new UpdateStateHelper(b).Execute(x, y), initialStateRdd);
state.ForeachRDD((time, rdd) => new DStreamTextFileSampleHelper().ForEachRDDHelper(time, rdd));
return context;
});
StartFileServer(ssc, directory, "words.txt");
ssc.Start();
ssc.AwaitTermination();
}
public class DStreamTextFileSampleHelper
{
public void ForEachRDDHelper(double time, RDD<dynamic> rdd)
{
// there's chance rdd.Take conflicts with ssc.Stop
if (stopFileServer)
return;
object[] taken = rdd.Take(10);
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Time: {0}", time);
Console.WriteLine("-------------------------------------------");
foreach (object record in taken)
{
Console.WriteLine(record);
var countByWord = (Tuple<string, int>)record;
Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "lazy" || countByWord.Item1 == "dog" ? 92 : 88);
}
Console.WriteLine();
stopFileServer = true;
}
}
private static string brokers = ConfigurationManager.AppSettings["KafkaTestBrokers"] ?? "127.0.0.1:9092";
private static string topic = ConfigurationManager.AppSettings["KafkaTestTopic"] ?? "test";
// expected partitions
private static int partitions = int.Parse(ConfigurationManager.AppSettings["KafkaTestPartitions"] ?? "10");
// total message count
private static uint messages = uint.Parse(ConfigurationManager.AppSettings["KafkaMessageCount"] ?? "100");
/// <summary>
/// start a local kafka service and create a 'test' topic with some data before running this sample
/// e.g. create 2 partitions with 100 messages which will be repartitioned into 10 RDD partitions
/// TODO: automate kafka service
/// </summary>
[Sample("experimental")]
internal static void DStreamDirectKafkaWithRepartitionSample()
{
count = 0;
string directory = SparkCLRSamples.Configuration.SampleDataLocation;
string checkpointPath = Path.Combine(directory, "checkpoint");
StreamingContext ssc = StreamingContext.GetOrCreate(checkpointPath,
() =>
{
var conf = new SparkConf();
SparkContext sc = new SparkContext(conf);
StreamingContext context = new StreamingContext(sc, 2000L);
context.Checkpoint(checkpointPath);
var kafkaParams = new List<Tuple<string, string>> {
new Tuple<string, string>("metadata.broker.list", brokers),
new Tuple<string, string>("auto.offset.reset", "smallest")
};
conf.Set("spark.mobius.streaming.kafka.numPartitions." + topic, partitions.ToString());
var dstream = KafkaUtils.CreateDirectStream(context, new List<string> { topic }, kafkaParams, Enumerable.Empty<Tuple<string, long>>());
dstream.ForeachRDD((time, rdd) => new DStreamDirectKafkaWithRepartitionSampleHelper().ForEachRddHelper(time, rdd));
return context;
});
ssc.Start();
ssc.AwaitTermination();
}
public class DStreamDirectKafkaWithRepartitionSampleHelper
{
public void ForEachRddHelper(double time, RDD<dynamic> rdd)
{
long batchCount = rdd.Count();
int numPartitions = rdd.GetNumPartitions();
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Time: {0}", time);
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Count: " + batchCount);
Console.WriteLine("Partitions: " + numPartitions);
// only first batch has data and is repartitioned into 10 partitions
if (count++ == 0)
{
Assert.AreEqual(messages, batchCount);
Assert.IsTrue(numPartitions >= partitions);
}
else
{
Assert.AreEqual(0, batchCount);
Assert.IsTrue(numPartitions == 0);
}
}
}
/// <summary>
/// A sample shows that ConstantInputDStream is an input stream that always returns the same mandatory input RDD at every batch time.
/// </summary>
[Sample("experimental")]
internal static void DStreamConstantDStreamSample()
{
var sc = SparkCLRSamples.SparkContext;
var ssc = new StreamingContext(sc, 2000L);
const int count = 100;
const int partitions = 2;
// create the RDD
var seedRDD = sc.Parallelize(Enumerable.Range(0, count), partitions);
var dstream = new ConstantInputDStream<int>(seedRDD, ssc);
dstream.ForeachRDD((time, rdd) => new DStreamConstantDStreamSampleHelper().ForEachRdd(time, rdd));
ssc.Start();
ssc.AwaitTermination();
}
public class DStreamConstantDStreamSampleHelper
{
public void ForEachRdd(double time, RDD<dynamic> rdd)
{
long batchCount = rdd.Count();
int numPartitions = rdd.GetNumPartitions();
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Time: {0}", time);
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Count: " + batchCount);
Console.WriteLine("Partitions: " + numPartitions);
Assert.AreEqual(count, batchCount);
Assert.AreEqual(partitions, numPartitions);
}
}
/// <summary>
/// when windowDuration not >= slideDuration * 5
/// DStreamReduceByKeyAndWindow does winodwed reduce once
/// </summary>
[Sample("experimental")]
internal static void DStreamReduceByKeyAndSmallWindowSample()
{
slideDuration = 6;
DStreamReduceByKeyAndWindowSample();
}
/// <summary>
/// when windowDuration >= slideDuration * 5
/// DStreamReduceByKeyAndWindow reduces twice based on previousRDD
/// by first invReduce on old RDDs and then reduce on new RDDs
/// </summary>
[Sample("experimental")]
internal static void DStreamReduceByKeyAndLargeWindowSample()
{
slideDuration = 4;
DStreamReduceByKeyAndWindowSample();
}
private static int slideDuration;
private static void DStreamReduceByKeyAndWindowSample()
{
count = 0;
const long bacthIntervalMs = 2000; // batch interval is in milliseconds
const int windowDuration = 26; // window duration in seconds
const int numPartitions = 2;
var sc = SparkCLRSamples.SparkContext;
var ssc = new StreamingContext(sc, bacthIntervalMs);
// create the RDD
var seedRDD = sc.Parallelize(Enumerable.Range(0, 100), numPartitions);
var numbers = new ConstantInputDStream<int>(seedRDD, ssc);
var pairs = numbers.Map(n => new Tuple<int, int>(n % numPartitions, n));
var reduced = pairs.ReduceByKeyAndWindow(
(int x, int y) => (x + y),
(int x, int y) => (x - y),
windowDuration,
slideDuration,
numPartitions
);
reduced.ForeachRDD((time, rdd) => new DStreamReduceByKeyAndLargeWindowSampleHelper(bacthIntervalMs, windowDuration, numPartitions).ForEachRdd(time, rdd));
ssc.Start();
ssc.AwaitTermination();
}
public class DStreamReduceByKeyAndLargeWindowSampleHelper
{
public long bacthIntervalMs; // batch interval is in milliseconds
public int windowDuration; // window duration in seconds
public int numPartitions;
public DStreamReduceByKeyAndLargeWindowSampleHelper(long bacthIntervalMs, int windowDuration, int numPartitions)
{
this.bacthIntervalMs = bacthIntervalMs;
this.windowDuration = windowDuration;
this.numPartitions = numPartitions;
}
public DStreamReduceByKeyAndLargeWindowSampleHelper() { }
public void ForEachRdd(double time, RDD<dynamic> rdd)
{
count++;
var taken = rdd.Collect();
int partitions = rdd.GetNumPartitions();
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Time: {0}", time);
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Batch: " + count);
Console.WriteLine("Count: " + taken.Length);
Console.WriteLine("Partitions: " + partitions);
Assert.AreEqual(taken.Length, 2);
Assert.AreEqual(partitions, numPartitions);
foreach (object record in taken)
{
Tuple<int, int> sum = (Tuple<int, int>)record;
Console.WriteLine("Key: {0}, Value: {1}", sum.Item1, sum.Item2);
// when batch count reaches window size, sum of even/odd number stay at windowDuration / slideDuration * (2450, 2500) respectively
Assert.AreEqual(sum.Item2, (count > windowDuration / slideDuration ? windowDuration : count * slideDuration) / (bacthIntervalMs / 1000) * (sum.Item1 == 0 ? 2450 : 2500));
}
}
}
[Sample("experimental")]
internal static void DStreamCSharpInputSample()
{
const int numPartitions = 5;
var sc = SparkCLRSamples.SparkContext;
var ssc = new StreamingContext(sc, 2000L); // batch interval is in milliseconds
var inputDStream = CSharpInputDStreamUtils.CreateStream<string>(
ssc,
numPartitions,
(double time, int pid) => DStreamCSharpInputSampleHelperFunc(time, pid));
inputDStream.ForeachRDD((time, rdd) => new DStreamCSharpInputSampleHelper().ForEachRdd(time, rdd));
ssc.Start();
ssc.AwaitTermination();
}
internal class DStreamCSharpInputSampleHelper
{
public void ForEachRdd(double time, RDD<dynamic> rdd)
{
var taken = rdd.Collect();
int partitions = rdd.GetNumPartitions();
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Time: {0}", time);
Console.WriteLine("-------------------------------------------");
Console.WriteLine("Count: " + taken.Length);
Console.WriteLine("Partitions: " + partitions);
foreach (object record in taken)
{
Console.WriteLine(record);
}
}
}
public static IEnumerable<string> DStreamCSharpInputSampleHelperFunc(double time, int pid)
{
var list = new List<string>() { string.Format("PluggableInputDStream-{0}-{1}", pid, time) };
return list.AsEnumerable();
}
}
// Use this helper class to test broacast variable in streaming application
[Serializable]
internal class UpdateStateHelper
{
private Broadcast<int> b;
internal UpdateStateHelper(Broadcast<int> b)
{
this.b = b;
}
internal int Execute(IEnumerable<Tuple<int, int>> vs, int s)
{
int result = vs.Sum(x => x.Item1 + x.Item2) + s + b.Value;
return result;
}
}
}