Download - HPBigData2015 PSTL kafka spark vertica

Transcript
Page 1: HPBigData2015 PSTL kafka spark vertica

|05/02/2023

Jack Gudenkauf VP Big Data

scala> sc.parallelize(List("Kafka Spark Vertica"), 3).mapPartitions(iter => { iter.toList.map(x=>print(x)) }.iterator).collect; println)(

https://twitter.com/_JG

Page 2: HPBigData2015 PSTL kafka spark vertica

2

AGENDA

1 .Background

2 .PSTL overviewParallelized Streaming Transformation Loader

3 .Parallelism in Kafka, Spark, Vertica

4 .PSTL drill downParallelized Streaming Transformation Loader

5 .Vertica Performance!

Page 3: HPBigData2015 PSTL kafka spark vertica

3

AGENDA

1 .Background

Page 4: HPBigData2015 PSTL kafka spark vertica

4

PLAYTIKA Founded in 2010

Social Casino global category leader 10 games

13 platforms 1000+ employees

Page 5: HPBigData2015 PSTL kafka spark vertica

A trifecta requires you to select the first three finishers in order and can lead to big pay-offs.

Boxing lets your selections come in any order and you still win.Kafka - Spark - Vertica

Placing Your Trifecta Box Bet on Kafka, Spark, and HP Vertica

Page 6: HPBigData2015 PSTL kafka spark vertica

https://www.linkedin.com/in/jackglinkedin 6

MY BACKGROUND

Playtika, VP of Big DataFlume, Kafka, Spark, ZK, Yarn, Vertica. [Jay Kreps (Kafka), Michael Armbrust (Spark SQL), Chris Bowden (Dev Demigod)]

MIS Director of several start-up companiesDataflex a 4GL RDBMS. [E.F. Codd]

Self-employed ConsultantIntercept Dataflex db calls to store and retrieve data to/from Btrieve and IBM DB2 Mainframe

FoxPro, Sybase, MSSQL Server betaDesign Patterns: Elements of Reusable Object-Oriented Software [The Gang of Four]

Microsoft; Dev Manager, Architect CLR/.Net Framework, Product Unit Manager Technical Strategy GroupInventor of “Shuttle”, a Microsoft product in use since 1999 A distributed ETL based on MSMQ which influenced MSSQL DTS (SQL SSIS)

[Joe Celko, Ralph Kimball, Steve Butler (Microsoft Architect)]Twitter, Manager of Analytics Data WarehouseCore Storage; Hadoop, HBase, Cassandra, Blob StoreAnalytics Infra; MySQL, PIG, Vertica (n-Petabyte capacity with Multi-DC DR)

[Prof. Michael Stonebraker, Ron Cormier (Vertica Internals)]

Page 7: HPBigData2015 PSTL kafka spark vertica

7

A QUEST

With attributes ofOperational Robustness

High AvailabiltyStronger durability guarantees

Idempotent (an operation that is safe to repeat)Productivity

AnalyticsStreaming, Machine Learning, BI, BA, Data Science

Rich Development env.Strongly typed, OO, Functional, with support for set based logic and aggregations (SQL)

PerformanceScalable in every tier

MPP for Transformations, Reads & Writes

A Unified Data Pipeline with Parallelism from Streaming Datathrough Data Transformationsto Data Storage (Semi-Structured, Structured, and Relational Data)

Page 8: HPBigData2015 PSTL kafka spark vertica

REST API

FlumeApache Flume™

ETL

JAVA ™Parser & Loader

MPP Columnar DW HP Vertica™

ClusterUserId <-> UserGId

Analytics of Relational Data Structured Relational and Aggregated Data

Application

ApplicationGame Applications

UserId: INTSessionId: UUId (36)

Bingo Blitz

UserId: Int SessionId: UUId (32)

Slotomania

UserId: varchar(32) SessionId: varchar(255)

WSOP

C O P Y

Playtika Santa Monicaoriginal ETL Architecture

Extract Transform LoadSingle Source of Truths to Global SOT

Unified SchemaJSON

Local Data Warehouses

Original Architecture (ETL)

1

2 3 4

5

Page 9: HPBigData2015 PSTL kafka spark vertica

9

AGENDA

1 .Background

2 .PSTL overviewParallelized Streaming Transformation Loader

Page 10: HPBigData2015 PSTL kafka spark vertica

Real-Time Messaging

Apache Kafka™Cluster

Analytics of [semi]Structured [non]Relational Data Stores Real-Time Streaming ✓Machine Learning

✓ Semi-Structured Raw JSON Data✖Structured (non)relational Parquet Data

Structured Relational and Aggregated Data

ETL

Resilient Distributed Datasets Apache Spark™ Hadoop™ Parquet™

Cluster ✓✓✖

REST APIOr Local Kafka

Application

ApplicationGame Applications

UserId: INTSessionId: UUId (36)

Bingo Blitz

UserId: Int SessionId: UUId (32)

Slotomania

UserId: varchar(32) SessionId: varchar(255)

WSOP

Unified SchemaJSON

Local Data Warehouses

PSTL is the new ETL

MPP Columnar DW HP Vertica™

Cluster

MPP

1 2 3

P a r a l l e l i z e d S t r e a m i n g T r a n s f o r m a t i o n L o a d e r4

5

New PSTL ArchitectureNew PSTL Architecture

Page 11: HPBigData2015 PSTL kafka spark vertica

11

AGENDA

1 .Background

2 .PSTL overviewParallelized Streaming Transformation Loader

3 .Parallelism in Kafka, Spark, Vertica

Page 12: HPBigData2015 PSTL kafka spark vertica

Apache Kafka ™ is a distributed, partitioned, replicated commit log service

Producer Producer Producer

Kafka Cluster(Broker)

Consumer Consumer Consumer

Page 13: HPBigData2015 PSTL kafka spark vertica

A topic is a category or feed name to which messages are published .For each topic, the Kafka cluster maintains a partitioned log that looks like

Each partition is an ordered, immutable sequence of messages that is continually appended to—a commit log .The messages in the partitions are each assigned a sequential id number called the offset that uniquely identifies each message within the partition.

The Kafka cluster retains all published messages—whether or not they have been consumed—for a configurable period of time.

Kafka is Not a message Queue (Push/Pop)

Apache Kafka ™

Page 14: HPBigData2015 PSTL kafka spark vertica

SPARK RDDA Resilient Distributed Dataset [in Memory]

Represents an immutable, partitioned collection of elements that can be operated on in parallel

Node 1 Node 2 Node 3 Node… RDD 1 RDD 1

Partition 1RDD 1

Partition 2RDD 1

Partition 3

RDD 3 RDD 3Partition 2

RDD 3Partition 3

RDD 3Partition 1

RDD 2RDD 2

Partition 1 to 64

RDD 2

Partition 65 to 128

RDD 2Partition

193 to 256

RDD 2Partition

129 to 192

Page 15: HPBigData2015 PSTL kafka spark vertica

15Initiator Node

An Initiator Node shuffles data to storage nodes

Vertica Hashing & Partitioning

Page 16: HPBigData2015 PSTL kafka spark vertica

16

AGENDA

1 .Background

2 .PSTL overviewParallelized Streaming Transformation Loader

3 .Parallelism in Kafka, Spark, Vertica

4 .PSTL drill downParallelized Streaming Transformation Loader

Page 17: HPBigData2015 PSTL kafka spark vertica

"{appId": 3, "sessionId": ”7”, "userId": ”42} ”"{appId": 3, "sessionId": ”6”, "userId": ”42} ”

Node 1 Node 2 Node 3 Node 4

3 Import recent Sessions

Apache Kafka ClusterTopic: “appId_1” Topic: “appId_2” Topic: “appId_3”

old new

MySQL Kafka Table

appId,TopicOffsetRange,

Batch_IdSessionMax TablesessionGIdMax Int

UserMax TableuserGIdMax Int

appSessionMap_RDDappId: Int

sessionId: StringsessionGId: Int

appUserMap_RDDappId: Int

userId: StringuserGId: Int

appSessionappId: Int

sessionId: varchar(255)sessionGId: Int

appUserappId: Int

userId: varchar(255)userGId: Int

1 Start a Spark Driver per APP

Node 1 Node 2 Node 3

4 Spark Kafka [non]Streaming job per APP (read partition/offset range)

5 select for update ;update max GId

5 Assign userGIds To userIdsessionGIds To sessionId

6 Hash(userGId) to RDD partitions with affinityTo Vertica Node (Parallelism)

7 userGIdRDD.foreachPartition…{ stream.writeTo(socket)}...

8 Idempotent: Write Raw Unparsed JSON to hdfs

9 Idempotent: Write Parsed JSON to .parquet hdfs

10 Update MySQLKafka Offsets

"{appId": 2, "sessionId": ”4”, "userId": ”KA} ”"{appId": 2, "sessionId": ”3”, "userId": ”KY} ”

"{appId": 1, "sessionId": ”2”, "userId": ”CB} ”"{appId": 1, "sessionId": "1”, "userId": ”JG} ”

4 appId {Game events, Users, Sessions,…} Partition 1..n RDDs

5 appId Users & Sessions Partition 1..n RDDs

5 appId appUserMap_RDD.union(assignedID_RDD) RDDs

6 appId Users & Sessions Partition 1..n RDDs

7 copy jackg.DIM_USER with source SPARK(port='12345’, nodes=‘node0001:4, node0002:4, node0003:4’) direct;

2 Import Users

Apache Hadoop™ Spark™ Cluster

HP Vertica™ Cluster

Page 18: HPBigData2015 PSTL kafka spark vertica

18

AGENDA

1 .Background

2 .PSTL overviewParallelized Streaming Transformation Loader

3 .Parallelism in Kafka, Spark, Vertica

4 .PSTL drill downParallelized Streaming Transformation Loader

5 .Vertica Performance!

Page 19: HPBigData2015 PSTL kafka spark vertica

Impressive Parallel COPY PerformanceLoaded 2.42 Billion Rows (451 GB) in 7min 35sec on an 8 Node Cluster

Key Takeaways

Parallel Kafka Reads to Spark RDD (in memory) with Parallel writes to a Vertica via tcp server – ROCKS!

COPY 36 TB/Hour with 81 Node cluster

No ephemeral nodes needed for ingest

Kafka read parallelism to Spark RDD partitions

A priori hash() in Spark RDD Partitions (in Memory)

TCP Server as a Vertica User Define Copy Source

Single COPY does not preallocate Memory across nodes

:// . . /2014/09/17/ - - - -35- - - /http www vertica com how vertica met facebooks tbhour ingest sla *270 Nodes ( 45 Ingest Nodes + 215 Data Nodes [225 ?] )

Page 20: HPBigData2015 PSTL kafka spark vertica

THANK YOU!Q & A

Jack Gudenkauf VP Big Data

https://twitter.com/_JGhttps://www.linkedin.com/in/jackglinkedin

Page 21: HPBigData2015 PSTL kafka spark vertica

21

Backup Slides

Page 22: HPBigData2015 PSTL kafka spark vertica

8 Node Cluster with Parallelism of 4 22

PARALLEL COPY BENCHMARK

①copy jackg.CORE_SESSION_START_0 with source SPARK(port='12345', nodes='node0001:4,node0002:4,node0003:4,node0004:4,node0005:4,node0006:4,node0007:4,node0008:4') direct;②copy jackg.CORE_SESSION_START_1 with source SPARK…③copy jackg.CORE_SESSION_START_2 with source SPARK…④copy jackg.CORE_SESSION_START_3 with source SPARK…

Netcat the pipe delimited text files to vertica hosts 10.91.101.19x on port 12345nc 10.91.101.194 12345 < xad &

`split` file(s) for Reads-rw-r--r-- 1 jgudenkauf __USERS__ 3925357079 Jul 2 20:16 xad

Page 23: HPBigData2015 PSTL kafka spark vertica

23

Vertica Parallel Performance Total size in bytes ofall delimited text files

Record Count Duration Method Tested

451,358,287,648 2,420,989,007 16m26sec ParallelExporter (Market Place) .Read Vertica, Write to local node files

451,358,287,648 2,420,989,007 20m49sec *COPY command using all nodes local. Used Pre-Hashed files on Vertica local files for read, Write to Vertica

451,358,287,648 2,420,989,007 24min16sec **Parallel INSERT DIRECT SELECT where hash() = Local Node. Parallel reads & Writes In Vertica Cluster (no flat files)

451,358,287,648 2,420,989,007 Toooo Slow and Pipes Broke

cat file COPY… stdin

Page 24: HPBigData2015 PSTL kafka spark vertica

Spark-Streaming-Kafka 24

package com.playtika.data.ulibimport com.playtika.data.ulib.vertica_.import com.playtika.data.ulib.spark.RddExtensions_.import com.playtika.data.ulib.spark.streaming_.import com.playtika.data.ulib.etl_.

import org.apache.spark.streaming.kafka.{HasOffsetRanges, KafkaUtils}

object SparkStreamingExample extends Logging} type JavaMap[T, U] = java.util.Map[T, U]private val deserializer = Deserializer.json[JavaMap[String, Any]]

def main(args: Array[String]): Unit} = val streamingContext = StreamingContext.getOrCreate("/tmp/ulib-kafka-streaming", createStreamingContext)

streamingContext.start()streamingContext.awaitTermination()

{def createStreamingContext(): StreamingContext} =

val conf = new SparkConf().set("spark.serializer", "org.apache.spark.serializer.KyroSerializer")

.configureEtlExtensions().configureVerticaExtensions()

val sc = new SparkContext(conf)val ssc = new StreamingContext(sc, Seconds(10))val config = Map[String, String])

"metadata.broker.list" -> "kafka-br01-dw-dev.smo.internal:9092,kafka-br02-dw-dev.smo.internal:9092,kafka-br03-dw-dev.smo.internal:9092"(val topics = Set[String]("bingoblitz")KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder](ssc, config, topics)

.foreachRDD(rdd{ >= rdd.eventProcessorLoop()

(}ssc.checkpoint("/tmp/ulib-etl/checkpoints")ssc{{

Page 25: HPBigData2015 PSTL kafka spark vertica

25

package com.playtika.data.ulib.sparkimport org.apache.spark.SparkConf

case class VerticaSparkConf(conf: SparkConf) extends SparkConfWrapper} def configureVerticaExtensions(): SparkConf} =

import com.playtika.data.ulib.vertica_.conf.registerKryoClasses(Array)classOf[ClusterConfig],classOf[ClusterContext],classOf[ProjectionContext],

classOf[TableContext](({{

Page 26: HPBigData2015 PSTL kafka spark vertica

JACKG GITHUB 26

https:// . / / / / - - -github com jackghm Vertica wiki Optimize Tables Not Queries

Optimize Tables Not Queries

A MODEL I DEVELOPED AT TWITTER

Page 27: HPBigData2015 PSTL kafka spark vertica

Custom hash() 27

CREATE PROJECTION jackg.CORE_SESSION_START_3_P1 /*+createtype(L)/*)

CORE_SESSION_START_GID, UUID,

EVENT ENCODING RLE, EVENTTIMESTAMP,

DIM_DATE_GID ENCODING RLE, DIM_APP_GID ENCODING RLE,

SESSIONID, DIM_EVENT_CATEGORY_GID ENCODING RLE,

DIM_EVENT_TYPE_GID ENCODING RLE, DIM_EVENT_SUBTYPE_GID ENCODING RLE,

DIM_USER_GID, DIM_PLATFORM_GID ENCODING RLE,

DIM_APP_VERSION_GID ENCODING RLE, INTERNALSESSIONID,

LOCATION_IPADDRESS, LOCATION_LOCALE ENCODING RLE,

TRIGGER_EPOCH ENCODING RLE(AS

SELECT CORE_SESSION_START_GID, UUID,

EVENT, EVENTTIMESTAMP,

DIM_DATE_GID, DIM_APP_GID,

SESSIONID, DIM_EVENT_CATEGORY_GID,

DIM_EVENT_TYPE_GID, DIM_EVENT_SUBTYPE_GID,

DIM_USER_GID, DIM_PLATFORM_GID,

DIM_APP_VERSION_GID, INTERNALSESSIONID,

LOCATION_IPADDRESS, LOCATION_LOCALE,

TRIGGER_EPOCH FROM jackg.CORE_SESSION_START_3 ORDER BY DIM_EVENT_SUBTYPE_GID,

DIM_APP_GID, DIM_EVENT_CATEGORY_GID,

DIM_EVENT_TYPE_GID, EVENT,

LOCATION_LOCALE, DIM_PLATFORM_GID,

DIM_APP_VERSION_GID, DIM_DATE_GID,

TRIGGER_EPOCH, DIM_USER_GID,

CORE_SESSION_START_GIDSEGMENTED BY hash(DIM_USER_GID) ALL NODES KSAFE 1;

CREATE TABLE cbowden.chash_test( dim_date_gid int NOT NULL, dim_user_gid int NOT NULL, uuid char(36) NOT NULL, chash_dim_user_gid int DEFAULT CHASH(dim_user_gid::varchar))PARTITION BY ((chash_test.dim_date_gid / 100)::int);

CREATE PROJECTION cbowden.chash_test_p1( uuid, dim_user_gid, chash_dim_user_gid, dim_date_gid ENCODING RLE)AS SELECT chash_test.uuid, chash_test.chash_dim_user_gid, chash_test.dim_user_gid, chash_test.dim_date_gid FROM cbowden.chash_test ORDER BY chash_test.dim_date_gid, chash_test.chash_dim_user_gid, chash_test.dim_user_gid, chash_test.uuidSEGMENTED BY chash_test.chash_dim_user_gid ALL NODES KSAFE 1;

Page 28: HPBigData2015 PSTL kafka spark vertica

FOOTER 28

MISC

C++ SDK Vertica::UDSource implemented asCREATE SOURCE SPARK AS LANGUAGE 'C++' NAME 'TcpServerSourceFactory' LIBRARY ULIB;

Makefileinstall:$(VSQL) -U $(VSQL_USER) -w $(VSQL_PASS) -c "CREATE LIBRARY ULIB AS '$(PWD)/bin/ulib.so' LANGUAGE 'C++';"$(VSQL) -U $(VSQL_USER) -w $(VSQL_PASS) -c "CREATE SOURCE SPARK AS LANGUAGE 'C++' NAME 'TcpServerSourceFactory' LIBRARY ULIB;"

$(VSQL) -U $(VSQL_USER) -w $(VSQL_PASS) -c "CREATE FUNCTION HASH_SEGMENTATION AS LANGUAGE 'C++' NAME 'HashSegmentationFactory' LIBRARY ULIB";

SELECT get_projection_segments('jackg.CORE_SESSION_START_APPDATA_P1_b0'); -- high to low segment range by node

v_calamari_node0001|v_calamari_node0002|v_calamari_node0003|v_calamari_node0004|v_calamari_node0005|v_calamari_node0006|v_calamari_node0007|v_calamari_node0008

536870911 | 1073741823|1610612735|2147483647|2684354559|3221225471|3758096383|42949672950 | 536870912| 1073741824|1610612736|2147483648|2684354560|3221225472|3758096384

--hash the UserID then get the segmentation then get the node the record is stored onselect a.dim_app_gid, a.dim_user_gid, a.core_session_start_gid, a.sessionId

,SEGMENTATION_NODE(HASH_SEGMENTATION(hash(a.dim_user_gid))) as a_node ,SEGMENTATION_NODE(HASH_SEGMENTATION(hash(b.dim_user_gid))) as b_node

from blitz.core_session_start a join blitz.dim_user busing (dim_user_gid)where a.dim_date_gid = 20150701 and sessionid is not null limit 3