SlideShare a Scribd company logo
1 of 37
Gimel Data Platform Overview
Agenda
©2018 PayPal Inc. Confidential and proprietary. 2
• Introduction
• PayPal’s Analytics Ecosystem
• Why Gimel
• Challenges in Analytics
• Walk through simple use case
• Gimel Open Source Journey
About Us
• Product manager, data processing products
at PayPal
• 20 years in data and analytics across
networking, semi-conductors, telecom,
security and fintech industries
• Data warehouse developer, BI program
manager, Data product manager
romehta@paypal.com
https://www.linkedin.com/in/romit-mehta/
©2018 PayPal Inc. Confidential and proprietary. 3
Romit Mehta
• Big data platform engineer at PayPal
• 13 years in data engineering, 5 years in
scalable solutions with big data
• Developed several Spark-based solutions
across NoSQL, Key-Value, Messaging,
Document based & relational systems
dmohanakumarchan@paypal.com
https://www.linkedin.com/in/deepakmc/
Deepak Mohanakumar Chandramouli
PayPal – Key Metrics and Analytics
Ecosystem
4©2018 PayPal Inc. Confidential and proprietary.
PayPal Big Data Platform
5
160+ PB Data
75,000+ YARN
jobs/day
One of the largest
Aerospike,
Teradata,
Hortonworks and
Oracle installations
Compute supported:
MR, Pig, Hive, Spark,
Beam
13 prod clusters, 12 non-
prod clusters
GPU co-located with
Hadoop
6
Developer Data scientist Analyst Operator
Gimel SDK Notebooks
PCatalog Data API
Infrastructure services leveraged for elasticity and redundancy
Multi-DC Public cloudPredictive resource allocation
Logging
Monitoring
Alerting
Security
Application
Lifecycle
Management
Compute
Frameworkand
APIs
GimelData
Platform
User
Experience
andAccess
R Studio BI tools
Why Gimel?
7
Use case - Flights Cancelled
9
Kafka Teradata External
HDFS / Hive
Data Prep / Availability
ProcessStream Ingest LoadExtract/Load
Parquet/ORC/Text?
Productionalize, Logging, Monitoring, Alerting, Auditing, Data Quality
Data SourcesData Points
Flights Events
Airports
Airlines
Carrier
Geography & Geo
Tags
Publish
Use case challenges
…
©2018 PayPal Inc. Confidential and proprietary.
Analysis
Real-time/
processed data
©2018 PayPal Inc. Confidential and proprietary. 10
Spark Read From Hbase
Data Access Code is Cumbersome and Fragile
©2018 PayPal Inc. Confidential and proprietary. 11
Spark Read From Hbase Spark Read From Elastic Search
Spark Read From AeroSpike Spark Read From Druid
Data Access Code is Cumbersome and Fragile
©2018 PayPal Inc. Confidential and proprietary. 12
Datasets Challenges
Data access tied
to compute and
data store
versions
Hard to find
available
data sets
Storage-specific
dataset creation
results in
duplication and
increased latency
No audit
trail for
dataset
access
No standards for
on-boarding data
sets for others to
discover
No statistics
on data set
usage and
access trends
Datasets
©2018 PayPal Inc. Confidential and proprietary. 13
High-friction Data Application Lifecycle
Learn Code Optimize Build Deploy RunOnboarding Big Data Apps
Learn Code Optimize Build Deploy RunCompute Engine Changed
Learn Code Optimize Build Deploy RunCompute Version Upgraded
Learn Code Optimize Build Deploy RunStorage API Changed
Learn Code Optimize Build Deploy RunStorage Connector Upgraded
Learn Code Optimize Build Deploy RunStorage Hosts Migrated
Learn Code Optimize Build Deploy RunStorage Changed
Learn Code Optimize Build Deploy Run*********************
Gimel Demo
14
15
API, PCatalog, Tools
With Gimel & Notebooks
©2018 PayPal Inc. Confidential and proprietary.
Kafka Teradata External
HDFS/ Hive
Data Prep / Availability
ProcessIngest LoadExtract/Load
Parquet/ORC/Text?
Productionalize, Logging, Monitoring, Alerting, Auditing, Data QC
Data SourcesData Points
Flights Events
Airports
Airlines
Carrier
Geography & Geo Tags
Analysis Publish
Use case challenges - Simplified with Gimel
©2018 PayPal Inc. Confidential and proprietary.
Spark Read From Hbase Spark Read From Elastic Search
Spark Read From AeroSpike Spark Read From Druid
With Data API
✔
Data Access Simplified with Gimel Data API
16
©2018 PayPal Inc. Confidential and proprietary.
Spark Read From Hbase Spark Read From Elastic Search
Spark Read From AeroSpike Spark Read From Druid
With Data API
✔
SQL Support in Gimel Data Platform
17
©2018 PayPal Inc. Confidential and proprietary. 18
Data Application Lifecycle with Data API
Learn Code Optimize Build Deploy RunOnboarding Big Data Apps
RunCompute Engine Changed
Compute Version Upgraded
Storage API Changed
Storage Connector Upgraded
Storage Hosts Migrated
Storage Changed
*********************
Run
Run
Run
Run
Run
Run
Open Source
19©2018 PayPal Inc. Confidential and proprietary.
Gimel Open Source Journey
• Open source Gimel
PCatalog:
• Metadata
services
• Discovery
services
• Catalog UI
• Open source Compute
Framework (SCaaS)
• Livy features and
enhancements
• Monitoring and
alerting
• SDK and Gimel
integration
• Open source PayPal
Notebooks
• Jupyter features
and enhancements
• Gimel integration
©2018 PayPal Inc. Confidential and proprietary.
• Open sourced Gimel
Data API in April 2018
(http://try.gimel.io)
Gimel - Open Sourced
Codebase available: https://github.com/paypal/gimel
Slack: https://gimel-dev.slack.com
Google Groups: https://groups.google.com/d/forum/gimel-dev
©2017 PayPal Inc. Confidential and proprietary. 21
Q&A
G i t h u b : h t t p : / / g i m e l . i o
Tr y i t y o u r s e l f : h t t p : / / t r y. g i m e l . i o
S l a c k : h t t p s : / / g i m e l - d e v. s l a c k . c o m
G o o g l e G r o u p s : h t t p s : / / g r o u p s . g o o g l e . c o m / d / f o r u m / g i m e l - d e v
22
Gimel – Deep Dive
23
Job
LIVY GRID
Job Server
Batch
Livy
API
NAS
Batch
In InIn
Interactive
Sparkling
Water
Interactive
Interactive
Metrics
History Server
Thrift Server
In InIn
Interactive
Interactive
Log
Log
Indexing
Search
xDiscovery
Maintain
Catalog
Scan
Discover
Metadata
Services
PCatalog UI
Explore
Configure
Log
Indexing
Search
PayPal Analytics Ecosystem
©2018 PayPal Inc. Confidential and proprietary.
©2018 PayPal Inc. Confidential and proprietary. 25
A peek into
Streaming SQL
Launches … Spark Streaming App
--StreamingWindowSeconds
setgimel.kafka.throttle.streaming.window.seconds=10;
--Throttling
setgimel.kafka.throttle.streaming.maxRatePerPartition=1500;
--ZK checkpoint rootpath
setgimel.kafka.consumer.checkpoint.root=/checkpoints/appname;
--Checkpoint enablingflag -implicitlycheckpoints aftereach mini-batch in streaming
setgimel.kafka.reader.checkpoint.save.enabled=true;
--Jupyter MagicforstreamingSQLon Notebooks | Interactive Usecases
--LivyREPL-Same magicforstreamingSQLworks | Streaming Usecases
%%gimel-stream
--AssumePre-SplitHBASETable as anexample
insertintopcatalog.HBASE_dataset
select
cust_id,
kafka_ds.*
frompcatalog.KAFKA_dataset kafka_ds;
Batch SQL
Launches … Spark Batch App
--Establish10 concurrent connections perTopic-Partition
setgimel.kafka.throttle.batch.parallelsPerPartition=10;
--Fetchat max-10 M messagesfromeach partition
setgimel.kafka.throttle.batch.maxRecordsPerPartition=10,000,000;
--Jupyter Magicon Notebooks | Interactive Usecases
--LivyREPL-Same magicworks| Batch Usecases
%%gimel
insertintopcatalog.HIVE_dataset
partition(yyyy,mm,dd,hh,mi)
selectkafka_ds.*,gimel_load_id
,substr(commit_timestamp,1,4)as yyyy
,substr(commit_timestamp,6,2)as mm
,substr(commit_timestamp,9,2)as dd
,substr(commit_timestamp,12,2)as hh
,case when cast(substr(commit_timestamp,15,2)asINT) <= 30then "00" else "30" end asmi
from pcatalog.KAFKA_dataset kafka_ds;
Following are Jupyter/Livy Magic terms
• %%gimel : calls gimel.executeBatch(sql)
• %%gimel-stream : calls
gimel.executeStream(sql)
gimel.dataset.factory {
KafkaDataSet
ElasticSearchDataSet
DruidDataSet
HiveDataSet
AerospikeDataSet
HbaseDataSet
CassandraDataSet
JDBCDataSet
}
Metadata
Services
dataSet.read(“dataSetName”,options)
dataSet.write(dataToWrite,”dataSetName”,options)
dataStream.read(“dataSetName”, options)
valstorageDataSet =getFromFactory(type=“Hive”)
{
Core Connector Implementation, example –Kafka
Combination ofOpen SourceConnector and
In-house implementations
Open source connector such asDataStax/SHC /ES-Spark
}
& Anatomy of API
gimel.datastream.factory{
KafkaDataStream
}
CatalogProvider.getDataSetProperties(“dataSetName”)
valstorageDataStream= getFromStreamFactory(type=“kafka”)
kafkaDataSet.read(“dataSetName”,options)
hiveDataSet.write(dataToWrite,”dataSetName”,options)
storageDataStream.read(“dataSetName”,options)
dataSet.write(”pcatalog.HIVE_dataset”,readDf, options)
val dataSet :gimel.DataSet =DataSet(sparkSession)
valdf1 =dataSet.read(“pcatalog.KAFKA_dataset”, options);
df1.createGlobalTempView(“tmp_abc123”)
Val resolvedSelectSQL= selectSQL.replace(“pcatalog.KAFKA_dataset”,”tmp_abc123”)
Val readDf : DataFrame= sparkSession.sql(resolvedSelectSQL);
selectkafka_ds.*,gimel_load_id
,substr(commit_timestamp,1,4)as yyyy
,substr(commit_timestamp,6,2)as mm
,substr(commit_timestamp,9,2)as dd
,substr(commit_timestamp,12,2)as hh
frompcatalog.KAFKA_dataset kafka_ds
join default.geo_lkp lkp
on kafka_ds.zip =geo_lkp.zip
where geo_lkp.region = ‘MIDWEST’
%%gimel
insertintopcatalog.HIVE_dataset
partition(yyyy,mm,dd,hh,mi)
--Establish10 concurrent connections perTopic-Partition
setgimel.kafka.throttle.batch.parallelsPerPartition=10;
--Fetch at max -10 M messagesfromeach partition
setgimel.kafka.throttle.batch.maxRecordsPerPartition=10,000,000;
©2018 PayPal Inc. Confidential and proprietary.
Setgimel.catalog.provider=PCATALOG
CatalogProvider.getDataSetProperties(“dataSetName”)
Metadata
Services
Setgimel.catalog.provider=USER
CatalogProvider.getDataSetProperties(“dataSetName”)
Setgimel.catalog.provider=HIVE
CatalogProvider.getDataSetProperties(“dataSetName”)
sql> set dataSetProperties={
"key.deserializer":"org.apache.kafka.common.serialization.StringDeserializer",
"auto.offset.reset":"earliest",
"gimel.kafka.checkpoint.zookeeper.host":"zookeeper:2181",
"gimel.storage.type":"kafka",
"gimel.kafka.whitelist.topics":"kafka_topic",
"datasetName":"test_table1",
"value.deserializer":"org.apache.kafka.common.serialization.ByteArrayDeserialize
r",
"value.serializer":"org.apache.kafka.common.serialization.ByteArraySerializer",
"gimel.kafka.checkpoint.zookeeper.path":"/pcatalog/kafka_consumer/checkpoint",
"gimel.kafka.avro.schema.source":"CSR",
"gimel.kafka.zookeeper.connection.timeout.ms":"10000",
"gimel.kafka.avro.schema.source.url":"http://schema_registry:8081",
"key.serializer":"org.apache.kafka.common.serialization.StringSerializer",
"gimel.kafka.avro.schema.source.wrapper.key":"schema_registry_key",
"gimel.kafka.bootstrap.servers":"localhost:9092"
}
sql> Select * from pcatalog.test_table1.
spark.sql("set gimel.catalog.provider=USER");
val dataSetOptions = DataSetProperties(
"KAFKA",
Array(Field("payload","string",true)) ,
Array(),
Map(
"datasetName" -> "test_table1",
"auto.offset.reset"-> "earliest",
"gimel.kafka.bootstrap.servers"-> "localhost:9092",
"gimel.kafka.avro.schema.source"-> "CSR",
"gimel.kafka.avro.schema.source.url"-> "http://schema_registry:8081",
"gimel.kafka.avro.schema.source.wrapper.key"-> "schema_registry_key",
"gimel.kafka.checkpoint.zookeeper.host"-> "zookeeper:2181",
"gimel.kafka.checkpoint.zookeeper.path"->
"/pcatalog/kafka_consumer/checkpoint",
"gimel.kafka.whitelist.topics"-> "kafka_topic",
"gimel.kafka.zookeeper.connection.timeout.ms"-> "10000",
"gimel.storage.type"-> "kafka",
"key.serializer"-> "org.apache.kafka.common.serialization.StringSerializer",
"value.serializer"-> "org.apache.kafka.common.serialization.ByteArraySerializer"
)
)
dataSet.read(”test_table1",Map("dataSetProperties"->dataSetOptions))
CREATE EXTERNAL TABLE `pcatalog.test_table1`
(payload string)
LOCATION 'hdfs://tmp/'
TBLPROPERTIES (
"datasetName" -> "dummy",
"auto.offset.reset"-> "earliest",
"gimel.kafka.bootstrap.servers"-> "localhost:9092",
"gimel.kafka.avro.schema.source"-> "CSR",
"gimel.kafka.avro.schema.source.url"-> "http://schema_registry:8081",
"gimel.kafka.avro.schema.source.wrapper.key"-> "schema_registry_key",
"gimel.kafka.checkpoint.zookeeper.host"-> "zookeeper:2181",
"gimel.kafka.checkpoint.zookeeper.path"->
"/pcatalog/kafka_consumer/checkpoint",
"gimel.kafka.whitelist.topics"-> "kafka_topic",
"gimel.kafka.zookeeper.connection.timeout.ms"-> "10000",
"gimel.storage.type"-> "kafka",
"key.serializer"-> "org.apache.kafka.common.serialization.StringSerializer",
"value.serializer"->
"org.apache.kafka.common.serialization.ByteArraySerializer"
);
Spark-sql> Select * from pcatalog.test_table1
Scala> dataSet.read(”test_table1",Map("dataSetProperties"-
>dataSetOptions))
Catalog Provider – USER | HIVE | PCATALOG | Your Own Catalog
Metadata
Setgimel.catalog.provider=YOUR_CATALOG
CatalogProvider.getDataSetProperties(“dataSetName”)
{
//Implement this!
}
©2018 PayPal Inc. Confidential and proprietary.
Spark Thrift Server
org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.sc
ala
//result = sqlContext.sql(statement)  Original SQL Execution
//Integration of Gimel in Spark
result = GimelQueryProcessor.executeBatch(statement, sqlContext.sparkSession)
Integration with ecosystems
class SparkSqlInterpreter(conf: SparkConf) extends SparkInterpreter(conf) {
private val SCALA_MAGIC = "%%[sS][cC][aA][lL][aA] (.*)".r
private val PCATALOG_BATCH_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".r
private val PCATALOG_STREAM_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".sS][tT][rR][eE][aA][mM] (.*)".r
// ........
// .....
case PCATALOG_BATCH_MAGIC(gimelCode) => GimelQueryProcessor.executeBatch(gimelCode,
sparkSession)
case PCATALOG_STREAM_MAGIC(gimelCode) => GimelQueryProcessor.executeStream(gimelCode,
sparkSession)
case _ =>
// ........
// .....
com/cloudera/livy/repl/SparkSqlInterpreter.scala
Livy REPL
sparkmagic/sparkmagic/kernels/sparkkernel/kernel.js
define(['base/js/namespace'], function(IPython){
var onload = function() {
IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-sql'] =
{'reg':[/^%%gimel/]};}
return { onload: onload }})
Jupyter Notebooks
©2018 PayPal Inc. Confidential and proprietary.
Data Stores Supported
©2018 PayPal Inc. Confidential and proprietary. 29
Systems
REST datasets
Acknowledgements
30
Acknowledgements
Gimel and PayPal Notebooks team:
Andrew Alves
Anisha Nainani
Ayushi Agarwal
Baskaran Gopalan
Dheeraj Rampally
Deepak Chandramouli
Laxmikant Patil
Meisam Fathi Salmi
Prabhu Kasinathan
Praveen Kanamarlapudi
Romit Mehta
Thilak Balasubramanian
Weijun Qian
31
Appendix
32©2018 PayPal Inc. Confidential and proprietary.
References Used
Images Referred :
https://www.google.com/search?q=big+data+stack+images&source=lnms&tbm=isch&sa=X&ved=0ahUKEwip1Jz3voPaAhU
oxFQKHV33AsgQ_AUICigB&biw=1440&bih=799
33©2018 PayPal Inc. Confidential and proprietary.
Spark Thrift Server - Integration
spark/sql/hive-
thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
//result = sqlContext.sql(statement)  Original SQL Execution
//Integration of Gimel in Spark
result = GimelQueryProcessor.executeBatch(statement, sqlContext.sparkSession)
©2018 PayPal Inc. Confidential and proprietary.
Livy - Integration
class SparkSqlInterpreter(conf: SparkConf) extends SparkInterpreter(conf) {
private val SCALA_MAGIC = "%%[sS][cC][aA][lL][aA] (.*)".r
private val PCATALOG_BATCH_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".r
private val PCATALOG_STREAM_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".sS][tT][rR][eE][aA][mM] (.*)".r
// ........
// .....
override def execute(code: String, outputPath: String): Interpreter.ExecuteResponse = {
require(sparkContext != null && sqlContext != null && sparkSession != null)
code match {
case SCALA_MAGIC(scalaCode) =>
super.execute(scalaCode, null)
case PCATALOG_BATCH_MAGIC(gimelCode) =>
Try {
GimelQueryProcessor.executeBatch(gimelCode, sparkSession)
} match {
case Success(x) => Interpreter.ExecuteSuccess(TEXT_PLAIN -> x)
case _ => Interpreter.ExecuteError("Failed", " ")
}
case PCATALOG_STREAM_MAGIC(gimelCode) =>
Try {
GimelQueryProcessor.executeStream(gimelCode, sparkSession)
} match {
case Success(x) => Interpreter.ExecuteSuccess(TEXT_PLAIN -> x)
case _ => Interpreter.ExecuteError("Failed", " ")
}
case _ =>
// ........
// .....
/repl/src/main/scala/com/cloudera/livy/repl/SparkSqlInterpreter.s
cala
©2018 PayPal Inc. Confidential and proprietary.
PayPal Notebooks (Jupyter) - Integration
def _scala_pcatalog_command(self, sql_context_variable_name):
if sql_context_variable_name == u'spark':
command = u'val output= {{import java.io.{{ByteArrayOutputStream, StringReader}};val outCapture = new
ByteArrayOutputStream;Console.withOut(outCapture){{gimel.GimelQueryProcessor.executeBatch("""{}""",sparkSession)}}}}'.format(self.query)
else:
command = u'val output= {{import java.io.{{ByteArrayOutputStream, StringReader}};val outCapture = new
ByteArrayOutputStream;Console.withOut(outCapture){{gimel..GimelQueryProcessor.executeBatch("""{}""",{})}}}}'.format(self.query, sql_context_variable_name)
if self.samplemethod == u'sample':
command = u'{}.sample(false, {})'.format(command, self.samplefraction)
if self.maxrows >= 0:
command = u'{}.take({})'.format(command, self.maxrows)
else:
command = u'{}.collect'.format(command)
return Command(u'{}.foreach(println)'.format(command+';noutput'))
sparkmagic/sparkmagic/livyclientlib/sqlquery.py
sparkmagic/sparkmagic/kernels/sparkkernel/kernel.js
define(['base/js/namespace'], function(IPython){
var onload = function() {
IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-sql'] =
{'reg':[/^%%sql/]};
IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-python'] =
{'reg':[/^%%local/]};
IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-sql'] =
{'reg':[/^%%gimel/]};}
return { onload: onload }
})
©2018 PayPal Inc. Confidential and proprietary.
Connectors | High level
©2018 PayPal Inc. Confidential and proprietary. 37
Storage Version API Implementation
Kafka 0.10.2 Batch & Stream Connectors – Implementation from scratch
Elastic Search 5.4.6 Connector | https://www.elastic.co/guide/en/elasticsearch/hadoop/5.4/spark.html
Additional implementations added in Gimel to support daily / monthly partitioned indexes in ES
Aerospike 3.1x Read | Aerospike Spark Connector(Aerospark) is used to read data directly into a DataFrame
(https://github.com/sasha-polev/aerospark)
Write | Aerospike Native Java Client Put API is used.
For each partition of the Dataframe a client connection is established, to write data from that partition to Aerospike.
HBASE 1.2 Connector | Horton Works HBASE Connector for Spark (SHC)
https://github.com/hortonworks-spark/shc
Cassandra 2.x Connector | DataStax Connector
https://github.com/datastax/spark-cassandra-connector
HIVE 1.2 Leverages spark APIs under the hood.
Druid 0.82 Connector | Leverages Tranquility under the hood
https://github.com/druid-io/tranquility
Teradata /
Relational
Leverages JDBC Storage Handler
Support for Batch Reads/Loads , FAST Load & FAST Exports
Alluxio Leverage Cross cluster access via reads using Spark Conf : spark.yarn.access.namenodes

More Related Content

What's hot

Highly configurable and extensible data processing framework at PubMatic
Highly configurable and extensible data processing framework at PubMaticHighly configurable and extensible data processing framework at PubMatic
Highly configurable and extensible data processing framework at PubMaticDataWorks Summit
 
Oracle Stream Analytics - Developer Introduction
Oracle Stream Analytics - Developer IntroductionOracle Stream Analytics - Developer Introduction
Oracle Stream Analytics - Developer IntroductionJeffrey T. Pollock
 
Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...
Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...
Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...Databricks
 
Gimel and PayPal Notebooks @ TDWI Leadership Summit Orlando
Gimel and PayPal Notebooks @ TDWI Leadership Summit OrlandoGimel and PayPal Notebooks @ TDWI Leadership Summit Orlando
Gimel and PayPal Notebooks @ TDWI Leadership Summit OrlandoRomit Mehta
 
PayPal Notebooks at Jupytercon 2018
PayPal Notebooks at Jupytercon 2018PayPal Notebooks at Jupytercon 2018
PayPal Notebooks at Jupytercon 2018Romit Mehta
 
The Convergence of Reporting and Interactive BI on Hadoop
The Convergence of Reporting and Interactive BI on HadoopThe Convergence of Reporting and Interactive BI on Hadoop
The Convergence of Reporting and Interactive BI on HadoopDataWorks Summit
 
2017 OpenWorld Keynote for Data Integration
2017 OpenWorld Keynote for Data Integration2017 OpenWorld Keynote for Data Integration
2017 OpenWorld Keynote for Data IntegrationJeffrey T. Pollock
 
Intelligent Integration OOW2017 - Jeff Pollock
Intelligent Integration OOW2017 - Jeff PollockIntelligent Integration OOW2017 - Jeff Pollock
Intelligent Integration OOW2017 - Jeff PollockJeffrey T. Pollock
 
IBM THINK 2018 - IBM Cloud SQL Query Introduction
IBM THINK 2018 - IBM Cloud SQL Query IntroductionIBM THINK 2018 - IBM Cloud SQL Query Introduction
IBM THINK 2018 - IBM Cloud SQL Query IntroductionTorsten Steinbach
 
Accelerating query processing with materialized views in Apache Hive
Accelerating query processing with materialized views in Apache HiveAccelerating query processing with materialized views in Apache Hive
Accelerating query processing with materialized views in Apache HiveDataWorks Summit
 
Security, ETL, BI & Analytics, and Software Integration
Security, ETL, BI & Analytics, and Software IntegrationSecurity, ETL, BI & Analytics, and Software Integration
Security, ETL, BI & Analytics, and Software IntegrationDataWorks Summit
 
Tapping into the Big Data Reservoir (CON7934)
Tapping into the Big Data Reservoir (CON7934)Tapping into the Big Data Reservoir (CON7934)
Tapping into the Big Data Reservoir (CON7934)Jeffrey T. Pollock
 
Securing and governing a multi-tenant data lake within the financial industry
Securing and governing a multi-tenant data lake within the financial industrySecuring and governing a multi-tenant data lake within the financial industry
Securing and governing a multi-tenant data lake within the financial industryDataWorks Summit
 
Microservices Patterns with GoldenGate
Microservices Patterns with GoldenGateMicroservices Patterns with GoldenGate
Microservices Patterns with GoldenGateJeffrey T. Pollock
 
Preparing Your Data for Cloud Analytics & AI/ML
Preparing Your Data for Cloud Analytics & AI/ML Preparing Your Data for Cloud Analytics & AI/ML
Preparing Your Data for Cloud Analytics & AI/ML Amazon Web Services
 
From BI Developer to Data Engineer with Oracle Analytics Cloud, Data Lake
From BI Developer to Data Engineer with Oracle Analytics Cloud, Data LakeFrom BI Developer to Data Engineer with Oracle Analytics Cloud, Data Lake
From BI Developer to Data Engineer with Oracle Analytics Cloud, Data LakeRittman Analytics
 
Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...
Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...
Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...Spark Summit
 
The convergence of reporting and interactive BI on Hadoop
The convergence of reporting and interactive BI on HadoopThe convergence of reporting and interactive BI on Hadoop
The convergence of reporting and interactive BI on HadoopDataWorks Summit
 
Adding structure to your streaming pipelines: moving from Spark streaming to ...
Adding structure to your streaming pipelines: moving from Spark streaming to ...Adding structure to your streaming pipelines: moving from Spark streaming to ...
Adding structure to your streaming pipelines: moving from Spark streaming to ...DataWorks Summit
 
Time to Talk about Data Mesh
Time to Talk about Data MeshTime to Talk about Data Mesh
Time to Talk about Data MeshLibbySchulze
 

What's hot (20)

Highly configurable and extensible data processing framework at PubMatic
Highly configurable and extensible data processing framework at PubMaticHighly configurable and extensible data processing framework at PubMatic
Highly configurable and extensible data processing framework at PubMatic
 
Oracle Stream Analytics - Developer Introduction
Oracle Stream Analytics - Developer IntroductionOracle Stream Analytics - Developer Introduction
Oracle Stream Analytics - Developer Introduction
 
Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...
Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...
Journey to Creating a 360 View of the Customer: Implementing Big Data Strateg...
 
Gimel and PayPal Notebooks @ TDWI Leadership Summit Orlando
Gimel and PayPal Notebooks @ TDWI Leadership Summit OrlandoGimel and PayPal Notebooks @ TDWI Leadership Summit Orlando
Gimel and PayPal Notebooks @ TDWI Leadership Summit Orlando
 
PayPal Notebooks at Jupytercon 2018
PayPal Notebooks at Jupytercon 2018PayPal Notebooks at Jupytercon 2018
PayPal Notebooks at Jupytercon 2018
 
The Convergence of Reporting and Interactive BI on Hadoop
The Convergence of Reporting and Interactive BI on HadoopThe Convergence of Reporting and Interactive BI on Hadoop
The Convergence of Reporting and Interactive BI on Hadoop
 
2017 OpenWorld Keynote for Data Integration
2017 OpenWorld Keynote for Data Integration2017 OpenWorld Keynote for Data Integration
2017 OpenWorld Keynote for Data Integration
 
Intelligent Integration OOW2017 - Jeff Pollock
Intelligent Integration OOW2017 - Jeff PollockIntelligent Integration OOW2017 - Jeff Pollock
Intelligent Integration OOW2017 - Jeff Pollock
 
IBM THINK 2018 - IBM Cloud SQL Query Introduction
IBM THINK 2018 - IBM Cloud SQL Query IntroductionIBM THINK 2018 - IBM Cloud SQL Query Introduction
IBM THINK 2018 - IBM Cloud SQL Query Introduction
 
Accelerating query processing with materialized views in Apache Hive
Accelerating query processing with materialized views in Apache HiveAccelerating query processing with materialized views in Apache Hive
Accelerating query processing with materialized views in Apache Hive
 
Security, ETL, BI & Analytics, and Software Integration
Security, ETL, BI & Analytics, and Software IntegrationSecurity, ETL, BI & Analytics, and Software Integration
Security, ETL, BI & Analytics, and Software Integration
 
Tapping into the Big Data Reservoir (CON7934)
Tapping into the Big Data Reservoir (CON7934)Tapping into the Big Data Reservoir (CON7934)
Tapping into the Big Data Reservoir (CON7934)
 
Securing and governing a multi-tenant data lake within the financial industry
Securing and governing a multi-tenant data lake within the financial industrySecuring and governing a multi-tenant data lake within the financial industry
Securing and governing a multi-tenant data lake within the financial industry
 
Microservices Patterns with GoldenGate
Microservices Patterns with GoldenGateMicroservices Patterns with GoldenGate
Microservices Patterns with GoldenGate
 
Preparing Your Data for Cloud Analytics & AI/ML
Preparing Your Data for Cloud Analytics & AI/ML Preparing Your Data for Cloud Analytics & AI/ML
Preparing Your Data for Cloud Analytics & AI/ML
 
From BI Developer to Data Engineer with Oracle Analytics Cloud, Data Lake
From BI Developer to Data Engineer with Oracle Analytics Cloud, Data LakeFrom BI Developer to Data Engineer with Oracle Analytics Cloud, Data Lake
From BI Developer to Data Engineer with Oracle Analytics Cloud, Data Lake
 
Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...
Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...
Regulatory Reporting of Asset Trading Using Apache Spark-(Sudipto Shankar Das...
 
The convergence of reporting and interactive BI on Hadoop
The convergence of reporting and interactive BI on HadoopThe convergence of reporting and interactive BI on Hadoop
The convergence of reporting and interactive BI on Hadoop
 
Adding structure to your streaming pipelines: moving from Spark streaming to ...
Adding structure to your streaming pipelines: moving from Spark streaming to ...Adding structure to your streaming pipelines: moving from Spark streaming to ...
Adding structure to your streaming pipelines: moving from Spark streaming to ...
 
Time to Talk about Data Mesh
Time to Talk about Data MeshTime to Talk about Data Mesh
Time to Talk about Data Mesh
 

Similar to Dataworks | 2018-06-20 | Gimel data platform

Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?
Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?
Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?SnapLogic
 
apidays LIVE Australia 2020 - Data with a Mission by Matt McLarty
apidays LIVE Australia 2020 -  Data with a Mission by Matt McLarty apidays LIVE Australia 2020 -  Data with a Mission by Matt McLarty
apidays LIVE Australia 2020 - Data with a Mission by Matt McLarty apidays
 
apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...
apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...
apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...apidays
 
apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...
apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...
apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...apidays
 
DEM07 Best Practices for Monitoring Amazon ECS Containers Launched with Fargate
DEM07 Best Practices for Monitoring Amazon ECS Containers Launched with FargateDEM07 Best Practices for Monitoring Amazon ECS Containers Launched with Fargate
DEM07 Best Practices for Monitoring Amazon ECS Containers Launched with FargateAmazon Web Services
 
Motadata - Unified Product Suite for IT Operations and Big Data Analytics
Motadata - Unified Product Suite for IT Operations and Big Data AnalyticsMotadata - Unified Product Suite for IT Operations and Big Data Analytics
Motadata - Unified Product Suite for IT Operations and Big Data Analyticsnovsela
 
Achieving digital transformation with Siebel CRM and Oracle Cloud
Achieving digital transformation with Siebel CRM and Oracle Cloud Achieving digital transformation with Siebel CRM and Oracle Cloud
Achieving digital transformation with Siebel CRM and Oracle Cloud Sonia Wadhwa
 
Ad hoc analytics with Cassandra and Spark
Ad hoc analytics with Cassandra and SparkAd hoc analytics with Cassandra and Spark
Ad hoc analytics with Cassandra and SparkMohammed Guller
 
Pivotal Big Data Suite: A Technical Overview
Pivotal Big Data Suite: A Technical OverviewPivotal Big Data Suite: A Technical Overview
Pivotal Big Data Suite: A Technical OverviewVMware Tanzu
 
How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...
How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...
How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...Amazon Web Services
 
Pivotal Digital Transformation Forum: Journey to Become a Data-Driven Enterprise
Pivotal Digital Transformation Forum: Journey to Become a Data-Driven EnterprisePivotal Digital Transformation Forum: Journey to Become a Data-Driven Enterprise
Pivotal Digital Transformation Forum: Journey to Become a Data-Driven EnterpriseVMware Tanzu
 
Why You Need Manageability Now More than Ever and How to Get It
Why You Need Manageability Now More than Ever and How to Get ItWhy You Need Manageability Now More than Ever and How to Get It
Why You Need Manageability Now More than Ever and How to Get ItGustavo Rene Antunez
 
Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...
Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...
Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...DataStax Academy
 
Office 365 Monitoring Best Practices
Office 365 Monitoring Best PracticesOffice 365 Monitoring Best Practices
Office 365 Monitoring Best PracticesThousandEyes
 
20181127 オラクル講演資料(DataRobot AI Experience Tokyo)
20181127 オラクル講演資料(DataRobot AI Experience Tokyo)20181127 オラクル講演資料(DataRobot AI Experience Tokyo)
20181127 オラクル講演資料(DataRobot AI Experience Tokyo)オラクルエンジニア通信
 
IICS_Capabilities.pptx
IICS_Capabilities.pptxIICS_Capabilities.pptx
IICS_Capabilities.pptxNandan Kumar
 
Data orchestration | 2020 | Alluxio | Gimel
Data orchestration | 2020 | Alluxio | GimelData orchestration | 2020 | Alluxio | Gimel
Data orchestration | 2020 | Alluxio | GimelDeepak Chandramouli
 
CodeCamp Iasi - Creating serverless data analytics system on GCP using BigQuery
CodeCamp Iasi - Creating serverless data analytics system on GCP using BigQueryCodeCamp Iasi - Creating serverless data analytics system on GCP using BigQuery
CodeCamp Iasi - Creating serverless data analytics system on GCP using BigQueryMárton Kodok
 

Similar to Dataworks | 2018-06-20 | Gimel data platform (20)

Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?
Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?
Intelligent data summit: Self-Service Big Data and AI/ML: Reality or Myth?
 
apidays LIVE Australia 2020 - Data with a Mission by Matt McLarty
apidays LIVE Australia 2020 -  Data with a Mission by Matt McLarty apidays LIVE Australia 2020 -  Data with a Mission by Matt McLarty
apidays LIVE Australia 2020 - Data with a Mission by Matt McLarty
 
apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...
apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...
apidays LIVE Paris - Data with a mission: a COVID-19 API case study by Matt M...
 
Cloud Native with Kyma
Cloud Native with KymaCloud Native with Kyma
Cloud Native with Kyma
 
Top 5 Lessons Learned in Deploying AI in the Real World
Top 5 Lessons Learned in Deploying AI in the Real WorldTop 5 Lessons Learned in Deploying AI in the Real World
Top 5 Lessons Learned in Deploying AI in the Real World
 
apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...
apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...
apidays LIVE New York 2021 - Simplify Open Policy Agent with Styra DAS by Tim...
 
DEM07 Best Practices for Monitoring Amazon ECS Containers Launched with Fargate
DEM07 Best Practices for Monitoring Amazon ECS Containers Launched with FargateDEM07 Best Practices for Monitoring Amazon ECS Containers Launched with Fargate
DEM07 Best Practices for Monitoring Amazon ECS Containers Launched with Fargate
 
Motadata - Unified Product Suite for IT Operations and Big Data Analytics
Motadata - Unified Product Suite for IT Operations and Big Data AnalyticsMotadata - Unified Product Suite for IT Operations and Big Data Analytics
Motadata - Unified Product Suite for IT Operations and Big Data Analytics
 
Achieving digital transformation with Siebel CRM and Oracle Cloud
Achieving digital transformation with Siebel CRM and Oracle Cloud Achieving digital transformation with Siebel CRM and Oracle Cloud
Achieving digital transformation with Siebel CRM and Oracle Cloud
 
Ad hoc analytics with Cassandra and Spark
Ad hoc analytics with Cassandra and SparkAd hoc analytics with Cassandra and Spark
Ad hoc analytics with Cassandra and Spark
 
Pivotal Big Data Suite: A Technical Overview
Pivotal Big Data Suite: A Technical OverviewPivotal Big Data Suite: A Technical Overview
Pivotal Big Data Suite: A Technical Overview
 
How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...
How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...
How Trek10 Uses Datadog's Distributed Tracing to Improve AWS Lambda Projects ...
 
Pivotal Digital Transformation Forum: Journey to Become a Data-Driven Enterprise
Pivotal Digital Transformation Forum: Journey to Become a Data-Driven EnterprisePivotal Digital Transformation Forum: Journey to Become a Data-Driven Enterprise
Pivotal Digital Transformation Forum: Journey to Become a Data-Driven Enterprise
 
Why You Need Manageability Now More than Ever and How to Get It
Why You Need Manageability Now More than Ever and How to Get ItWhy You Need Manageability Now More than Ever and How to Get It
Why You Need Manageability Now More than Ever and How to Get It
 
Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...
Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...
Glassbeam: Ad-hoc Analytics on Internet of Complex Things with Apache Cassand...
 
Office 365 Monitoring Best Practices
Office 365 Monitoring Best PracticesOffice 365 Monitoring Best Practices
Office 365 Monitoring Best Practices
 
20181127 オラクル講演資料(DataRobot AI Experience Tokyo)
20181127 オラクル講演資料(DataRobot AI Experience Tokyo)20181127 オラクル講演資料(DataRobot AI Experience Tokyo)
20181127 オラクル講演資料(DataRobot AI Experience Tokyo)
 
IICS_Capabilities.pptx
IICS_Capabilities.pptxIICS_Capabilities.pptx
IICS_Capabilities.pptx
 
Data orchestration | 2020 | Alluxio | Gimel
Data orchestration | 2020 | Alluxio | GimelData orchestration | 2020 | Alluxio | Gimel
Data orchestration | 2020 | Alluxio | Gimel
 
CodeCamp Iasi - Creating serverless data analytics system on GCP using BigQuery
CodeCamp Iasi - Creating serverless data analytics system on GCP using BigQueryCodeCamp Iasi - Creating serverless data analytics system on GCP using BigQuery
CodeCamp Iasi - Creating serverless data analytics system on GCP using BigQuery
 

Recently uploaded

Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Miguel Araújo
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century educationjfdjdjcjdnsjd
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)wesley chun
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationRadu Cotescu
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherRemote DBA Services
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...Martijn de Jong
 
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfThe Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfEnterprise Knowledge
 
CNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of ServiceCNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of Servicegiselly40
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Enterprise Knowledge
 
What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?Antenna Manufacturer Coco
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slidevu2urc
 
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEarley Information Science
 
Handwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsHandwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsMaria Levchenko
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityPrincipled Technologies
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdfhans926745
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationMichael W. Hawkins
 
GenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdfGenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdflior mazor
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreternaman860154
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking MenDelhi Call girls
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsEnterprise Knowledge
 

Recently uploaded (20)

Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century education
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organization
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfThe Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
 
CNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of ServiceCNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of Service
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...
 
What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
 
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
 
Handwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsHandwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed texts
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day Presentation
 
GenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdfGenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdf
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreter
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
 

Dataworks | 2018-06-20 | Gimel data platform

  • 2. Agenda ©2018 PayPal Inc. Confidential and proprietary. 2 • Introduction • PayPal’s Analytics Ecosystem • Why Gimel • Challenges in Analytics • Walk through simple use case • Gimel Open Source Journey
  • 3. About Us • Product manager, data processing products at PayPal • 20 years in data and analytics across networking, semi-conductors, telecom, security and fintech industries • Data warehouse developer, BI program manager, Data product manager romehta@paypal.com https://www.linkedin.com/in/romit-mehta/ ©2018 PayPal Inc. Confidential and proprietary. 3 Romit Mehta • Big data platform engineer at PayPal • 13 years in data engineering, 5 years in scalable solutions with big data • Developed several Spark-based solutions across NoSQL, Key-Value, Messaging, Document based & relational systems dmohanakumarchan@paypal.com https://www.linkedin.com/in/deepakmc/ Deepak Mohanakumar Chandramouli
  • 4. PayPal – Key Metrics and Analytics Ecosystem 4©2018 PayPal Inc. Confidential and proprietary.
  • 5. PayPal Big Data Platform 5 160+ PB Data 75,000+ YARN jobs/day One of the largest Aerospike, Teradata, Hortonworks and Oracle installations Compute supported: MR, Pig, Hive, Spark, Beam 13 prod clusters, 12 non- prod clusters GPU co-located with Hadoop
  • 6. 6 Developer Data scientist Analyst Operator Gimel SDK Notebooks PCatalog Data API Infrastructure services leveraged for elasticity and redundancy Multi-DC Public cloudPredictive resource allocation Logging Monitoring Alerting Security Application Lifecycle Management Compute Frameworkand APIs GimelData Platform User Experience andAccess R Studio BI tools
  • 8. Use case - Flights Cancelled
  • 9. 9 Kafka Teradata External HDFS / Hive Data Prep / Availability ProcessStream Ingest LoadExtract/Load Parquet/ORC/Text? Productionalize, Logging, Monitoring, Alerting, Auditing, Data Quality Data SourcesData Points Flights Events Airports Airlines Carrier Geography & Geo Tags Publish Use case challenges … ©2018 PayPal Inc. Confidential and proprietary. Analysis Real-time/ processed data
  • 10. ©2018 PayPal Inc. Confidential and proprietary. 10 Spark Read From Hbase Data Access Code is Cumbersome and Fragile
  • 11. ©2018 PayPal Inc. Confidential and proprietary. 11 Spark Read From Hbase Spark Read From Elastic Search Spark Read From AeroSpike Spark Read From Druid Data Access Code is Cumbersome and Fragile
  • 12. ©2018 PayPal Inc. Confidential and proprietary. 12 Datasets Challenges Data access tied to compute and data store versions Hard to find available data sets Storage-specific dataset creation results in duplication and increased latency No audit trail for dataset access No standards for on-boarding data sets for others to discover No statistics on data set usage and access trends Datasets
  • 13. ©2018 PayPal Inc. Confidential and proprietary. 13 High-friction Data Application Lifecycle Learn Code Optimize Build Deploy RunOnboarding Big Data Apps Learn Code Optimize Build Deploy RunCompute Engine Changed Learn Code Optimize Build Deploy RunCompute Version Upgraded Learn Code Optimize Build Deploy RunStorage API Changed Learn Code Optimize Build Deploy RunStorage Connector Upgraded Learn Code Optimize Build Deploy RunStorage Hosts Migrated Learn Code Optimize Build Deploy RunStorage Changed Learn Code Optimize Build Deploy Run*********************
  • 15. 15 API, PCatalog, Tools With Gimel & Notebooks ©2018 PayPal Inc. Confidential and proprietary. Kafka Teradata External HDFS/ Hive Data Prep / Availability ProcessIngest LoadExtract/Load Parquet/ORC/Text? Productionalize, Logging, Monitoring, Alerting, Auditing, Data QC Data SourcesData Points Flights Events Airports Airlines Carrier Geography & Geo Tags Analysis Publish Use case challenges - Simplified with Gimel
  • 16. ©2018 PayPal Inc. Confidential and proprietary. Spark Read From Hbase Spark Read From Elastic Search Spark Read From AeroSpike Spark Read From Druid With Data API ✔ Data Access Simplified with Gimel Data API 16
  • 17. ©2018 PayPal Inc. Confidential and proprietary. Spark Read From Hbase Spark Read From Elastic Search Spark Read From AeroSpike Spark Read From Druid With Data API ✔ SQL Support in Gimel Data Platform 17
  • 18. ©2018 PayPal Inc. Confidential and proprietary. 18 Data Application Lifecycle with Data API Learn Code Optimize Build Deploy RunOnboarding Big Data Apps RunCompute Engine Changed Compute Version Upgraded Storage API Changed Storage Connector Upgraded Storage Hosts Migrated Storage Changed ********************* Run Run Run Run Run Run
  • 19. Open Source 19©2018 PayPal Inc. Confidential and proprietary.
  • 20. Gimel Open Source Journey • Open source Gimel PCatalog: • Metadata services • Discovery services • Catalog UI • Open source Compute Framework (SCaaS) • Livy features and enhancements • Monitoring and alerting • SDK and Gimel integration • Open source PayPal Notebooks • Jupyter features and enhancements • Gimel integration ©2018 PayPal Inc. Confidential and proprietary. • Open sourced Gimel Data API in April 2018 (http://try.gimel.io)
  • 21. Gimel - Open Sourced Codebase available: https://github.com/paypal/gimel Slack: https://gimel-dev.slack.com Google Groups: https://groups.google.com/d/forum/gimel-dev ©2017 PayPal Inc. Confidential and proprietary. 21
  • 22. Q&A G i t h u b : h t t p : / / g i m e l . i o Tr y i t y o u r s e l f : h t t p : / / t r y. g i m e l . i o S l a c k : h t t p s : / / g i m e l - d e v. s l a c k . c o m G o o g l e G r o u p s : h t t p s : / / g r o u p s . g o o g l e . c o m / d / f o r u m / g i m e l - d e v 22
  • 23. Gimel – Deep Dive 23
  • 24. Job LIVY GRID Job Server Batch Livy API NAS Batch In InIn Interactive Sparkling Water Interactive Interactive Metrics History Server Thrift Server In InIn Interactive Interactive Log Log Indexing Search xDiscovery Maintain Catalog Scan Discover Metadata Services PCatalog UI Explore Configure Log Indexing Search PayPal Analytics Ecosystem ©2018 PayPal Inc. Confidential and proprietary.
  • 25. ©2018 PayPal Inc. Confidential and proprietary. 25 A peek into Streaming SQL Launches … Spark Streaming App --StreamingWindowSeconds setgimel.kafka.throttle.streaming.window.seconds=10; --Throttling setgimel.kafka.throttle.streaming.maxRatePerPartition=1500; --ZK checkpoint rootpath setgimel.kafka.consumer.checkpoint.root=/checkpoints/appname; --Checkpoint enablingflag -implicitlycheckpoints aftereach mini-batch in streaming setgimel.kafka.reader.checkpoint.save.enabled=true; --Jupyter MagicforstreamingSQLon Notebooks | Interactive Usecases --LivyREPL-Same magicforstreamingSQLworks | Streaming Usecases %%gimel-stream --AssumePre-SplitHBASETable as anexample insertintopcatalog.HBASE_dataset select cust_id, kafka_ds.* frompcatalog.KAFKA_dataset kafka_ds; Batch SQL Launches … Spark Batch App --Establish10 concurrent connections perTopic-Partition setgimel.kafka.throttle.batch.parallelsPerPartition=10; --Fetchat max-10 M messagesfromeach partition setgimel.kafka.throttle.batch.maxRecordsPerPartition=10,000,000; --Jupyter Magicon Notebooks | Interactive Usecases --LivyREPL-Same magicworks| Batch Usecases %%gimel insertintopcatalog.HIVE_dataset partition(yyyy,mm,dd,hh,mi) selectkafka_ds.*,gimel_load_id ,substr(commit_timestamp,1,4)as yyyy ,substr(commit_timestamp,6,2)as mm ,substr(commit_timestamp,9,2)as dd ,substr(commit_timestamp,12,2)as hh ,case when cast(substr(commit_timestamp,15,2)asINT) <= 30then "00" else "30" end asmi from pcatalog.KAFKA_dataset kafka_ds; Following are Jupyter/Livy Magic terms • %%gimel : calls gimel.executeBatch(sql) • %%gimel-stream : calls gimel.executeStream(sql)
  • 26. gimel.dataset.factory { KafkaDataSet ElasticSearchDataSet DruidDataSet HiveDataSet AerospikeDataSet HbaseDataSet CassandraDataSet JDBCDataSet } Metadata Services dataSet.read(“dataSetName”,options) dataSet.write(dataToWrite,”dataSetName”,options) dataStream.read(“dataSetName”, options) valstorageDataSet =getFromFactory(type=“Hive”) { Core Connector Implementation, example –Kafka Combination ofOpen SourceConnector and In-house implementations Open source connector such asDataStax/SHC /ES-Spark } & Anatomy of API gimel.datastream.factory{ KafkaDataStream } CatalogProvider.getDataSetProperties(“dataSetName”) valstorageDataStream= getFromStreamFactory(type=“kafka”) kafkaDataSet.read(“dataSetName”,options) hiveDataSet.write(dataToWrite,”dataSetName”,options) storageDataStream.read(“dataSetName”,options) dataSet.write(”pcatalog.HIVE_dataset”,readDf, options) val dataSet :gimel.DataSet =DataSet(sparkSession) valdf1 =dataSet.read(“pcatalog.KAFKA_dataset”, options); df1.createGlobalTempView(“tmp_abc123”) Val resolvedSelectSQL= selectSQL.replace(“pcatalog.KAFKA_dataset”,”tmp_abc123”) Val readDf : DataFrame= sparkSession.sql(resolvedSelectSQL); selectkafka_ds.*,gimel_load_id ,substr(commit_timestamp,1,4)as yyyy ,substr(commit_timestamp,6,2)as mm ,substr(commit_timestamp,9,2)as dd ,substr(commit_timestamp,12,2)as hh frompcatalog.KAFKA_dataset kafka_ds join default.geo_lkp lkp on kafka_ds.zip =geo_lkp.zip where geo_lkp.region = ‘MIDWEST’ %%gimel insertintopcatalog.HIVE_dataset partition(yyyy,mm,dd,hh,mi) --Establish10 concurrent connections perTopic-Partition setgimel.kafka.throttle.batch.parallelsPerPartition=10; --Fetch at max -10 M messagesfromeach partition setgimel.kafka.throttle.batch.maxRecordsPerPartition=10,000,000; ©2018 PayPal Inc. Confidential and proprietary.
  • 27. Setgimel.catalog.provider=PCATALOG CatalogProvider.getDataSetProperties(“dataSetName”) Metadata Services Setgimel.catalog.provider=USER CatalogProvider.getDataSetProperties(“dataSetName”) Setgimel.catalog.provider=HIVE CatalogProvider.getDataSetProperties(“dataSetName”) sql> set dataSetProperties={ "key.deserializer":"org.apache.kafka.common.serialization.StringDeserializer", "auto.offset.reset":"earliest", "gimel.kafka.checkpoint.zookeeper.host":"zookeeper:2181", "gimel.storage.type":"kafka", "gimel.kafka.whitelist.topics":"kafka_topic", "datasetName":"test_table1", "value.deserializer":"org.apache.kafka.common.serialization.ByteArrayDeserialize r", "value.serializer":"org.apache.kafka.common.serialization.ByteArraySerializer", "gimel.kafka.checkpoint.zookeeper.path":"/pcatalog/kafka_consumer/checkpoint", "gimel.kafka.avro.schema.source":"CSR", "gimel.kafka.zookeeper.connection.timeout.ms":"10000", "gimel.kafka.avro.schema.source.url":"http://schema_registry:8081", "key.serializer":"org.apache.kafka.common.serialization.StringSerializer", "gimel.kafka.avro.schema.source.wrapper.key":"schema_registry_key", "gimel.kafka.bootstrap.servers":"localhost:9092" } sql> Select * from pcatalog.test_table1. spark.sql("set gimel.catalog.provider=USER"); val dataSetOptions = DataSetProperties( "KAFKA", Array(Field("payload","string",true)) , Array(), Map( "datasetName" -> "test_table1", "auto.offset.reset"-> "earliest", "gimel.kafka.bootstrap.servers"-> "localhost:9092", "gimel.kafka.avro.schema.source"-> "CSR", "gimel.kafka.avro.schema.source.url"-> "http://schema_registry:8081", "gimel.kafka.avro.schema.source.wrapper.key"-> "schema_registry_key", "gimel.kafka.checkpoint.zookeeper.host"-> "zookeeper:2181", "gimel.kafka.checkpoint.zookeeper.path"-> "/pcatalog/kafka_consumer/checkpoint", "gimel.kafka.whitelist.topics"-> "kafka_topic", "gimel.kafka.zookeeper.connection.timeout.ms"-> "10000", "gimel.storage.type"-> "kafka", "key.serializer"-> "org.apache.kafka.common.serialization.StringSerializer", "value.serializer"-> "org.apache.kafka.common.serialization.ByteArraySerializer" ) ) dataSet.read(”test_table1",Map("dataSetProperties"->dataSetOptions)) CREATE EXTERNAL TABLE `pcatalog.test_table1` (payload string) LOCATION 'hdfs://tmp/' TBLPROPERTIES ( "datasetName" -> "dummy", "auto.offset.reset"-> "earliest", "gimel.kafka.bootstrap.servers"-> "localhost:9092", "gimel.kafka.avro.schema.source"-> "CSR", "gimel.kafka.avro.schema.source.url"-> "http://schema_registry:8081", "gimel.kafka.avro.schema.source.wrapper.key"-> "schema_registry_key", "gimel.kafka.checkpoint.zookeeper.host"-> "zookeeper:2181", "gimel.kafka.checkpoint.zookeeper.path"-> "/pcatalog/kafka_consumer/checkpoint", "gimel.kafka.whitelist.topics"-> "kafka_topic", "gimel.kafka.zookeeper.connection.timeout.ms"-> "10000", "gimel.storage.type"-> "kafka", "key.serializer"-> "org.apache.kafka.common.serialization.StringSerializer", "value.serializer"-> "org.apache.kafka.common.serialization.ByteArraySerializer" ); Spark-sql> Select * from pcatalog.test_table1 Scala> dataSet.read(”test_table1",Map("dataSetProperties"- >dataSetOptions)) Catalog Provider – USER | HIVE | PCATALOG | Your Own Catalog Metadata Setgimel.catalog.provider=YOUR_CATALOG CatalogProvider.getDataSetProperties(“dataSetName”) { //Implement this! } ©2018 PayPal Inc. Confidential and proprietary.
  • 28. Spark Thrift Server org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.sc ala //result = sqlContext.sql(statement)  Original SQL Execution //Integration of Gimel in Spark result = GimelQueryProcessor.executeBatch(statement, sqlContext.sparkSession) Integration with ecosystems class SparkSqlInterpreter(conf: SparkConf) extends SparkInterpreter(conf) { private val SCALA_MAGIC = "%%[sS][cC][aA][lL][aA] (.*)".r private val PCATALOG_BATCH_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".r private val PCATALOG_STREAM_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".sS][tT][rR][eE][aA][mM] (.*)".r // ........ // ..... case PCATALOG_BATCH_MAGIC(gimelCode) => GimelQueryProcessor.executeBatch(gimelCode, sparkSession) case PCATALOG_STREAM_MAGIC(gimelCode) => GimelQueryProcessor.executeStream(gimelCode, sparkSession) case _ => // ........ // ..... com/cloudera/livy/repl/SparkSqlInterpreter.scala Livy REPL sparkmagic/sparkmagic/kernels/sparkkernel/kernel.js define(['base/js/namespace'], function(IPython){ var onload = function() { IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-sql'] = {'reg':[/^%%gimel/]};} return { onload: onload }}) Jupyter Notebooks ©2018 PayPal Inc. Confidential and proprietary.
  • 29. Data Stores Supported ©2018 PayPal Inc. Confidential and proprietary. 29 Systems REST datasets
  • 31. Acknowledgements Gimel and PayPal Notebooks team: Andrew Alves Anisha Nainani Ayushi Agarwal Baskaran Gopalan Dheeraj Rampally Deepak Chandramouli Laxmikant Patil Meisam Fathi Salmi Prabhu Kasinathan Praveen Kanamarlapudi Romit Mehta Thilak Balasubramanian Weijun Qian 31
  • 32. Appendix 32©2018 PayPal Inc. Confidential and proprietary.
  • 33. References Used Images Referred : https://www.google.com/search?q=big+data+stack+images&source=lnms&tbm=isch&sa=X&ved=0ahUKEwip1Jz3voPaAhU oxFQKHV33AsgQ_AUICigB&biw=1440&bih=799 33©2018 PayPal Inc. Confidential and proprietary.
  • 34. Spark Thrift Server - Integration spark/sql/hive- thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala //result = sqlContext.sql(statement)  Original SQL Execution //Integration of Gimel in Spark result = GimelQueryProcessor.executeBatch(statement, sqlContext.sparkSession) ©2018 PayPal Inc. Confidential and proprietary.
  • 35. Livy - Integration class SparkSqlInterpreter(conf: SparkConf) extends SparkInterpreter(conf) { private val SCALA_MAGIC = "%%[sS][cC][aA][lL][aA] (.*)".r private val PCATALOG_BATCH_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".r private val PCATALOG_STREAM_MAGIC = "%%[gG][iI][mM][eE][lL](.*)".sS][tT][rR][eE][aA][mM] (.*)".r // ........ // ..... override def execute(code: String, outputPath: String): Interpreter.ExecuteResponse = { require(sparkContext != null && sqlContext != null && sparkSession != null) code match { case SCALA_MAGIC(scalaCode) => super.execute(scalaCode, null) case PCATALOG_BATCH_MAGIC(gimelCode) => Try { GimelQueryProcessor.executeBatch(gimelCode, sparkSession) } match { case Success(x) => Interpreter.ExecuteSuccess(TEXT_PLAIN -> x) case _ => Interpreter.ExecuteError("Failed", " ") } case PCATALOG_STREAM_MAGIC(gimelCode) => Try { GimelQueryProcessor.executeStream(gimelCode, sparkSession) } match { case Success(x) => Interpreter.ExecuteSuccess(TEXT_PLAIN -> x) case _ => Interpreter.ExecuteError("Failed", " ") } case _ => // ........ // ..... /repl/src/main/scala/com/cloudera/livy/repl/SparkSqlInterpreter.s cala ©2018 PayPal Inc. Confidential and proprietary.
  • 36. PayPal Notebooks (Jupyter) - Integration def _scala_pcatalog_command(self, sql_context_variable_name): if sql_context_variable_name == u'spark': command = u'val output= {{import java.io.{{ByteArrayOutputStream, StringReader}};val outCapture = new ByteArrayOutputStream;Console.withOut(outCapture){{gimel.GimelQueryProcessor.executeBatch("""{}""",sparkSession)}}}}'.format(self.query) else: command = u'val output= {{import java.io.{{ByteArrayOutputStream, StringReader}};val outCapture = new ByteArrayOutputStream;Console.withOut(outCapture){{gimel..GimelQueryProcessor.executeBatch("""{}""",{})}}}}'.format(self.query, sql_context_variable_name) if self.samplemethod == u'sample': command = u'{}.sample(false, {})'.format(command, self.samplefraction) if self.maxrows >= 0: command = u'{}.take({})'.format(command, self.maxrows) else: command = u'{}.collect'.format(command) return Command(u'{}.foreach(println)'.format(command+';noutput')) sparkmagic/sparkmagic/livyclientlib/sqlquery.py sparkmagic/sparkmagic/kernels/sparkkernel/kernel.js define(['base/js/namespace'], function(IPython){ var onload = function() { IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-sql'] = {'reg':[/^%%sql/]}; IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-python'] = {'reg':[/^%%local/]}; IPython.CodeCell.config_defaults.highlight_modes['magic_text/x-sql'] = {'reg':[/^%%gimel/]};} return { onload: onload } }) ©2018 PayPal Inc. Confidential and proprietary.
  • 37. Connectors | High level ©2018 PayPal Inc. Confidential and proprietary. 37 Storage Version API Implementation Kafka 0.10.2 Batch & Stream Connectors – Implementation from scratch Elastic Search 5.4.6 Connector | https://www.elastic.co/guide/en/elasticsearch/hadoop/5.4/spark.html Additional implementations added in Gimel to support daily / monthly partitioned indexes in ES Aerospike 3.1x Read | Aerospike Spark Connector(Aerospark) is used to read data directly into a DataFrame (https://github.com/sasha-polev/aerospark) Write | Aerospike Native Java Client Put API is used. For each partition of the Dataframe a client connection is established, to write data from that partition to Aerospike. HBASE 1.2 Connector | Horton Works HBASE Connector for Spark (SHC) https://github.com/hortonworks-spark/shc Cassandra 2.x Connector | DataStax Connector https://github.com/datastax/spark-cassandra-connector HIVE 1.2 Leverages spark APIs under the hood. Druid 0.82 Connector | Leverages Tranquility under the hood https://github.com/druid-io/tranquility Teradata / Relational Leverages JDBC Storage Handler Support for Batch Reads/Loads , FAST Load & FAST Exports Alluxio Leverage Cross cluster access via reads using Spark Conf : spark.yarn.access.namenodes