SlideShare a Scribd company logo
1 of 51
Download to read offline
Data Analytics Service Company
and Its Ruby Usage
RubyKaigi 2015 (Dec 12, 2015)
Satoshi Tagomori (@tagomoris)
Satoshi "Moris" Tagomori
(@tagomoris)
Fluentd, MessagePack-Ruby, Norikra, ...
Treasure Data, Inc.
http://www.treasuredata.com/
http://www.fluentd.org/
Fluentd
Unified Logging Layer
For Stream Data
Written in CRuby
http://www.slideshare.net/treasure-data/the-basics-of-fluentd-35681111
Bulk Data Loader
High Throughput&Reliability
Embulk
Written in Java/JRuby
http://www.slideshare.net/frsyuki/embuk-making-data-integration-works-relaxed
http://www.embulk.org/
Data Analytics Platform
Data Analytics Service
Services
Services
JVM
Services
JVMC++
Data Analytics Flow
Collect Store Process Visualize
Data source
Reporting
Monitoring
Data Analytics Flow
Collect Store Process Visualize
Data source
Reporting
Monitoring
Data Analytics Platform
• Data collection, storage
• Console & API endpoints
• Schema management
• Processing (batch, query, ...)
• Queuing & Scheduling
• Data connector/exporter
Treasure Data Internals
Data Analytics Platform
• Data collection, storage: Ruby(OSS), Java/JRuby(OSS)
• Console & API endpoints: Ruby(RoR)
• Schema management: Ruby/Java (MessagePack)
• Processing (batch, query, ...): Java(Hadoop,Presto)
• Queuing & Scheduling: Ruby(OSS)
• Data connector/exporter: Java, Java/JRuby(OSS)
Treasure Data Architecture: Overview
http://www.slideshare.net/tagomoris/data-analytics-service-company-and-its-ruby-usage
Console
API
EventCollector
PlazmaDB
Worker
Hadoop
Cluster
Presto
Cluster
USERS
TD SDKs
SERVERS
DataConnector
CUSTOMER's
SYSTEMS
Scheduler
Console
API
EventCollector
PlazmaDB
Worker
Scheduler
Hadoop
Cluster
Presto
Cluster
USERS
TD SDKs
SERVERS
DataConnector
CUSTOMER's
SYSTEMS
Console
API
EventCollector
PlazmaDB
Worker
Scheduler
Hadoop
Cluster
Presto
Cluster
USERS
TD SDKs
SERVERS
DataConnector
CUSTOMER's
SYSTEMS
50k/day
200k/day
Console
API
EventCollector
PlazmaDB
Worker
Scheduler
Hadoop
Cluster
Presto
Cluster
USERS
TD SDKs
SERVERS
DataConnector
CUSTOMER's
SYSTEMS
50k/day
200k/day
12M/day
(138/sec)
Queue/Worker, Scheduler
• Treasure Data: multi-tenant data analytics service
• executes many jobs in shared clusters (queries,
imports, ...)
• CORE: queues-workers & schedulers
• Clusters have queues/scheduler... it's not enough
• resource limitations for each price plans
• priority queues for job types
• and many others
PerfectSched
https://github.com/treasure-data/perfectsched
PerfectSched
• Provides periodical/scheduled queries for customers
• it's like reliable "cron"
• Highly available distributed scheduler using RDBMS
• Written in CRuby
• At-least-once semantics
• PerfectSched enqueues jobs into PerfectQueue
Jobs in TD
LOST Duplicated
Retried
for
Errors
Throughput
Execution
time
DATA
import/
export
NG NG
OK
or
NG
HIGH SHORT
(secs-mins)
QUERY NG OK OK LOW
SHORT
(secs)
or
LONG
(mins-hours)
PerfectQueue
https://github.com/treasure-data/perfectqueue
Worker
Worker
Worker
Worker
PerfectQueue overview
Worker
MySQL
1 table for 1 queue
workers for queues
hypervisor process
worker processes
worker processes
worker processes
Features
• Priorities for query types
• Resource limits per accounts
• Graceful restarts
• Queries must run long time (<= 1d)
• New worker code should be loaded, besides
running job with older code
PerfectQueue
• Highly available distributed queue using RDBMS
• Enqueue by INSERT INTO
• Dequeue/Commit by UPDATE
• Using transactions
• Flexible scheduling rather than scalability
• Workers does many things
• Plazmadb operations (including importing data)
• Building job parameters
• Handling results of jobs + kicking other jobs
• Using Amazon RDS (MySQL) internally (+ Workers on EC2)
Builing Jobs/Parameters
• Parameters
• for job types, accounts, price plans and clusters
• to control performance/parallelism, permissions
and data types
• ex: Java properties
• Jobs
• to prepare for customers' queries
• to make queries safer/faster
• ex: Hive Queries (HiveQL, a variety of SQL)
Example: Hive job
env HADOOP_CLASSPATH=test.jar:td-hadoop-1.0.jar 
HADOOP_OPTS="-Xmx738m -Duser.name=221" 
hive --service jar td-hadoop-1.0.jar 
com.treasure_data.hadoop.hive.runner.QueryRunner 
-hiveconf td.jar.version= 
-hiveconf plazma.metadb.config={} 
-hiveconf plazma.storage.config={} 
-hiveconf td.worker.database.config={} 
-hiveconf mapreduce.job.priority=HIGH 
-hiveconf mapreduce.job.queuename=root.q221.high 
-hiveconf mapreduce.job.name=HiveJob379515 
-hiveconf td.query.mergeThreshold=1333382400 
-hiveconf td.query.apikey=12345 
-hiveconf td.scheduled.time=1342449253 
-hiveconf td.outdir=./jobs/379515 
-hiveconf hive.metastore.warehouse.dir=/user/hive/221/warehouse 
-hiveconf hive.auto.convert.join.noconditionaltask=false 
-hiveconf hive.mapjoin.localtask.max.memory.usage=0.7 
-hiveconf hive.mapjoin.smalltable.filesize=25000000 
-hiveconf hive.resultset.use.unique.column.names=false 
-hiveconf hive.auto.convert.join=false 
-hiveconf hive.optimize.sort.dynamic.partition=false 
-hiveconf mapreduce.job.reduces=-1 
-hiveconf hive.vectorized.execution.enabled=false 
-hiveconf mapreduce.job.ubertask.enable=true 
-hiveconf yarn.app.mapreduce.am.resource.mb=2048
env HADOOP_CLASSPATH=test.jar:td-hadoop-1.0.jar 
HADOOP_OPTS="-Xmx738m -Duser.name=221" 
hive --service jar td-hadoop-1.0.jar 
com.treasure_data.hadoop.hive.runner.QueryRunner 
-hiveconf td.jar.version= 
-hiveconf plazma.metadb.config={} 
-hiveconf plazma.storage.config={} 
-hiveconf td.worker.database.config={} 
-hiveconf mapreduce.job.priority=HIGH 
-hiveconf mapreduce.job.queuename=root.q221.high 
-hiveconf mapreduce.job.name=HiveJob379515 
-hiveconf td.query.mergeThreshold=1333382400 
-hiveconf td.query.apikey=12345 
-hiveconf td.scheduled.time=1342449253 
-hiveconf td.outdir=./jobs/379515 
-hiveconf hive.metastore.warehouse.dir=/user/hive/221/warehouse 
-hiveconf hive.auto.convert.join.noconditionaltask=false 
-hiveconf hive.mapjoin.localtask.max.memory.usage=0.7 
-hiveconf hive.mapjoin.smalltable.filesize=25000000 
-hiveconf hive.resultset.use.unique.column.names=false 
-hiveconf hive.auto.convert.join=false 
-hiveconf hive.optimize.sort.dynamic.partition=false 
-hiveconf mapreduce.job.reduces=-1 
-hiveconf hive.vectorized.execution.enabled=false 
-hiveconf mapreduce.job.ubertask.enable=true 
-hiveconf yarn.app.mapreduce.am.resource.mb=2048 
-hiveconf mapreduce.job.ubertask.maxmaps=1 
-hiveconf mapreduce.job.ubertask.maxreduces=1 
-hiveconf mapreduce.job.ubertask.maxbytes=536870912 
-hiveconf td.hive.insertInto.dynamic.partitioning=false 
-outdir ./jobs/379515
Example: Hive job (cont)
ADD JAR 'td-hadoop-1.0.jar';
CREATE DATABASE IF NOT EXISTS `db`;
USE `db`;
CREATE TABLE tagomoris (`v` MAP<STRING,STRING>, `time` INT)
STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler'
WITH SERDEPROPERTIES ('msgpack.columns.mapping'='*,time')
TBLPROPERTIES (
'td.storage.user'='221',
'td.storage.database'='dfc',
'td.storage.table'='users_20100604_080812_ce9203d0',
'td.storage.path'='221/dfc/users_20100604_080812_ce9203d0',
'td.table_id'='2',
'td.modifiable'='true',
'plazma.data_set.name'='221/dfc/users_20100604_080812_ce9203d0'
);
CREATE TABLE tbl1 (
`uid` INT,
`key` STRING,
`time` INT
)
STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler'
WITH SERDEPROPERTIES ('msgpack.columns.mapping'='uid,key,time')
TBLPROPERTIES (
'td.storage.user'='221',
'td.storage.database'='dfc',
ADD JAR 'td-hadoop-1.0.jar';
CREATE DATABASE IF NOT EXISTS `db`;
USE `db`;
CREATE TABLE tagomoris (`v` MAP<STRING,STRING>, `time` INT)
STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler'
WITH SERDEPROPERTIES ('msgpack.columns.mapping'='*,time')
TBLPROPERTIES (
'td.storage.user'='221',
'td.storage.database'='dfc',
'td.storage.table'='users_20100604_080812_ce9203d0',
'td.storage.path'='221/dfc/users_20100604_080812_ce9203d0',
'td.table_id'='2',
'td.modifiable'='true',
'plazma.data_set.name'='221/dfc/users_20100604_080812_ce9203d0'
);
CREATE TABLE tbl1 (
`uid` INT,
`key` STRING,
`time` INT
)
STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler'
WITH SERDEPROPERTIES ('msgpack.columns.mapping'='uid,key,time')
TBLPROPERTIES (
'td.storage.user'='221',
'td.storage.database'='dfc',
'td.storage.table'='contests_20100606_120720_96abe81a',
'td.storage.path'='221/dfc/contests_20100606_120720_96abe81a',
'td.table_id'='4',
'td.modifiable'='true',
'plazma.data_set.name'='221/dfc/contests_20100606_120720_96abe81a'
);
USE `db`;
USE `db`;
CREATE TEMPORARY FUNCTION MSGPACK_SERIALIZE AS
'com.treasure_data.hadoop.hive.udf.MessagePackSerialize';
CREATE TEMPORARY FUNCTION TD_TIME_RANGE AS
'com.treasure_data.hadoop.hive.udf.GenericUDFTimeRange';
CREATE TEMPORARY FUNCTION TD_TIME_ADD AS
'com.treasure_data.hadoop.hive.udf.UDFTimeAdd';
CREATE TEMPORARY FUNCTION TD_TIME_FORMAT AS
'com.treasure_data.hadoop.hive.udf.UDFTimeFormat';
CREATE TEMPORARY FUNCTION TD_TIME_PARSE AS
'com.treasure_data.hadoop.hive.udf.UDFTimeParse';
CREATE TEMPORARY FUNCTION TD_SCHEDULED_TIME AS
'com.treasure_data.hadoop.hive.udf.GenericUDFScheduledTime';
CREATE TEMPORARY FUNCTION TD_X_RANK AS
'com.treasure_data.hadoop.hive.udf.Rank';
CREATE TEMPORARY FUNCTION TD_FIRST AS
'com.treasure_data.hadoop.hive.udf.GenericUDAFFirst';
CREATE TEMPORARY FUNCTION TD_LAST AS
'com.treasure_data.hadoop.hive.udf.GenericUDAFLast';
CREATE TEMPORARY FUNCTION TD_SESSIONIZE AS
'com.treasure_data.hadoop.hive.udf.UDFSessionize';
CREATE TEMPORARY FUNCTION TD_PARSE_USER_AGENT AS
'com.treasure_data.hadoop.hive.udf.GenericUDFParseUserAgent';
CREATE TEMPORARY FUNCTION TD_HEX2NUM AS
'com.treasure_data.hadoop.hive.udf.UDFHex2num';
CREATE TEMPORARY FUNCTION TD_MD5 AS
'com.treasure_data.hadoop.hive.udf.UDFmd5';
CREATE TEMPORARY FUNCTION TD_RANK_SEQUENCE AS
'com.treasure_data.hadoop.hive.udf.UDFRankSequence';
CREATE TEMPORARY FUNCTION TD_STRING_EXPLODER AS
'com.treasure_data.hadoop.hive.udf.GenericUDTFStringExploder';
CREATE TEMPORARY FUNCTION TD_URL_DECODE AS
CREATE TEMPORARY FUNCTION TD_URL_DECODE AS
'com.treasure_data.hadoop.hive.udf.UDFUrlDecode';
CREATE TEMPORARY FUNCTION TD_DATE_TRUNC AS
'com.treasure_data.hadoop.hive.udf.UDFDateTrunc';
CREATE TEMPORARY FUNCTION TD_LAT_LONG_TO_COUNTRY AS
'com.treasure_data.hadoop.hive.udf.UDFLatLongToCountry';
CREATE TEMPORARY FUNCTION TD_SUBSTRING_INENCODING AS
'com.treasure_data.hadoop.hive.udf.GenericUDFSubstringInEncoding';
CREATE TEMPORARY FUNCTION TD_DIVIDE AS
'com.treasure_data.hadoop.hive.udf.GenericUDFDivide';
CREATE TEMPORARY FUNCTION TD_SUMIF AS
'com.treasure_data.hadoop.hive.udf.GenericUDAFSumIf';
CREATE TEMPORARY FUNCTION TD_AVGIF AS
'com.treasure_data.hadoop.hive.udf.GenericUDAFAvgIf';
CREATE TEMPORARY FUNCTION hivemall_version AS
'hivemall.HivemallVersionUDF';
CREATE TEMPORARY FUNCTION perceptron AS
'hivemall.classifier.PerceptronUDTF';
CREATE TEMPORARY FUNCTION train_perceptron AS
'hivemall.classifier.PerceptronUDTF';
CREATE TEMPORARY FUNCTION train_pa AS
'hivemall.classifier.PassiveAggressiveUDTF';
CREATE TEMPORARY FUNCTION train_pa1 AS
'hivemall.classifier.PassiveAggressiveUDTF';
CREATE TEMPORARY FUNCTION train_pa2 AS
'hivemall.classifier.PassiveAggressiveUDTF';
CREATE TEMPORARY FUNCTION train_cw AS
'hivemall.classifier.ConfidenceWeightedUDTF';
CREATE TEMPORARY FUNCTION train_arow AS
'hivemall.classifier.AROWClassifierUDTF';
CREATE TEMPORARY FUNCTION train_arowh AS
'hivemall.classifier.AROWClassifierUDTF';
CREATE TEMPORARY FUNCTION train_arowh AS
'hivemall.classifier.AROWClassifierUDTF';
CREATE TEMPORARY FUNCTION train_scw AS
'hivemall.classifier.SoftConfideceWeightedUDTF';
CREATE TEMPORARY FUNCTION train_scw2 AS
'hivemall.classifier.SoftConfideceWeightedUDTF';
CREATE TEMPORARY FUNCTION adagrad_rda AS
'hivemall.classifier.AdaGradRDAUDTF';
CREATE TEMPORARY FUNCTION train_adagrad_rda AS
'hivemall.classifier.AdaGradRDAUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_perceptron AS
'hivemall.classifier.multiclass.MulticlassPerceptronUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_pa AS
'hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_pa1 AS
'hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_pa2 AS
'hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_cw AS
'hivemall.classifier.multiclass.MulticlassConfidenceWeightedUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_arow AS
'hivemall.classifier.multiclass.MulticlassAROWClassifierUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_scw AS
'hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF';
CREATE TEMPORARY FUNCTION train_multiclass_scw2 AS
'hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF';
CREATE TEMPORARY FUNCTION cosine_similarity AS
'hivemall.knn.similarity.CosineSimilarityUDF';
CREATE TEMPORARY FUNCTION cosine_sim AS
'hivemall.knn.similarity.CosineSimilarityUDF';
CREATE TEMPORARY FUNCTION jaccard AS
'hivemall.knn.similarity.JaccardIndexUDF';
CREATE TEMPORARY FUNCTION jaccard AS
'hivemall.knn.similarity.JaccardIndexUDF';
CREATE TEMPORARY FUNCTION jaccard_similarity AS
'hivemall.knn.similarity.JaccardIndexUDF';
CREATE TEMPORARY FUNCTION angular_similarity AS
'hivemall.knn.similarity.AngularSimilarityUDF';
CREATE TEMPORARY FUNCTION euclid_similarity AS
'hivemall.knn.similarity.EuclidSimilarity';
CREATE TEMPORARY FUNCTION distance2similarity AS
'hivemall.knn.similarity.Distance2SimilarityUDF';
CREATE TEMPORARY FUNCTION hamming_distance AS
'hivemall.knn.distance.HammingDistanceUDF';
CREATE TEMPORARY FUNCTION popcnt AS 'hivemall.knn.distance.PopcountUDF';
CREATE TEMPORARY FUNCTION kld AS 'hivemall.knn.distance.KLDivergenceUDF';
CREATE TEMPORARY FUNCTION euclid_distance AS
'hivemall.knn.distance.EuclidDistanceUDF';
CREATE TEMPORARY FUNCTION cosine_distance AS
'hivemall.knn.distance.CosineDistanceUDF';
CREATE TEMPORARY FUNCTION angular_distance AS
'hivemall.knn.distance.AngularDistanceUDF';
CREATE TEMPORARY FUNCTION jaccard_distance AS
'hivemall.knn.distance.JaccardDistanceUDF';
CREATE TEMPORARY FUNCTION manhattan_distance AS
'hivemall.knn.distance.ManhattanDistanceUDF';
CREATE TEMPORARY FUNCTION minkowski_distance AS
'hivemall.knn.distance.MinkowskiDistanceUDF';
CREATE TEMPORARY FUNCTION minhashes AS 'hivemall.knn.lsh.MinHashesUDF';
CREATE TEMPORARY FUNCTION minhash AS 'hivemall.knn.lsh.MinHashUDTF';
CREATE TEMPORARY FUNCTION bbit_minhash AS
'hivemall.knn.lsh.bBitMinHashUDF';
CREATE TEMPORARY FUNCTION voted_avg AS
'hivemall.ensemble.bagging.VotedAvgUDAF';
CREATE TEMPORARY FUNCTION voted_avg AS
'hivemall.ensemble.bagging.VotedAvgUDAF';
CREATE TEMPORARY FUNCTION weight_voted_avg AS
'hivemall.ensemble.bagging.WeightVotedAvgUDAF';
CREATE TEMPORARY FUNCTION wvoted_avg AS
'hivemall.ensemble.bagging.WeightVotedAvgUDAF';
CREATE TEMPORARY FUNCTION max_label AS
'hivemall.ensemble.MaxValueLabelUDAF';
CREATE TEMPORARY FUNCTION maxrow AS 'hivemall.ensemble.MaxRowUDAF';
CREATE TEMPORARY FUNCTION argmin_kld AS
'hivemall.ensemble.ArgminKLDistanceUDAF';
CREATE TEMPORARY FUNCTION mhash AS
'hivemall.ftvec.hashing.MurmurHash3UDF';
CREATE TEMPORARY FUNCTION sha1 AS 'hivemall.ftvec.hashing.Sha1UDF';
CREATE TEMPORARY FUNCTION array_hash_values AS
'hivemall.ftvec.hashing.ArrayHashValuesUDF';
CREATE TEMPORARY FUNCTION prefixed_hash_values AS
'hivemall.ftvec.hashing.ArrayPrefixedHashValuesUDF';
CREATE TEMPORARY FUNCTION polynomial_features AS
'hivemall.ftvec.pairing.PolynomialFeaturesUDF';
CREATE TEMPORARY FUNCTION powered_features AS
'hivemall.ftvec.pairing.PoweredFeaturesUDF';
CREATE TEMPORARY FUNCTION rescale AS 'hivemall.ftvec.scaling.RescaleUDF';
CREATE TEMPORARY FUNCTION rescale_fv AS
'hivemall.ftvec.scaling.RescaleUDF';
CREATE TEMPORARY FUNCTION zscore AS 'hivemall.ftvec.scaling.ZScoreUDF';
CREATE TEMPORARY FUNCTION normalize AS
'hivemall.ftvec.scaling.L2NormalizationUDF';
CREATE TEMPORARY FUNCTION conv2dense AS
'hivemall.ftvec.conv.ConvertToDenseModelUDAF';
CREATE TEMPORARY FUNCTION to_dense_features AS
'hivemall.ftvec.conv.ToDenseFeaturesUDF';
CREATE TEMPORARY FUNCTION to_dense_features AS
'hivemall.ftvec.conv.ToDenseFeaturesUDF';
CREATE TEMPORARY FUNCTION to_dense AS
'hivemall.ftvec.conv.ToDenseFeaturesUDF';
CREATE TEMPORARY FUNCTION to_sparse_features AS
'hivemall.ftvec.conv.ToSparseFeaturesUDF';
CREATE TEMPORARY FUNCTION to_sparse AS
'hivemall.ftvec.conv.ToSparseFeaturesUDF';
CREATE TEMPORARY FUNCTION quantify AS
'hivemall.ftvec.conv.QuantifyColumnsUDTF';
CREATE TEMPORARY FUNCTION vectorize_features AS
'hivemall.ftvec.trans.VectorizeFeaturesUDF';
CREATE TEMPORARY FUNCTION categorical_features AS
'hivemall.ftvec.trans.CategoricalFeaturesUDF';
CREATE TEMPORARY FUNCTION indexed_features AS
'hivemall.ftvec.trans.IndexedFeatures';
CREATE TEMPORARY FUNCTION quantified_features AS
'hivemall.ftvec.trans.QuantifiedFeaturesUDTF';
CREATE TEMPORARY FUNCTION quantitative_features AS
'hivemall.ftvec.trans.QuantitativeFeaturesUDF';
CREATE TEMPORARY FUNCTION amplify AS
'hivemall.ftvec.amplify.AmplifierUDTF';
CREATE TEMPORARY FUNCTION rand_amplify AS
'hivemall.ftvec.amplify.RandomAmplifierUDTF';
CREATE TEMPORARY FUNCTION addBias AS 'hivemall.ftvec.AddBiasUDF';
CREATE TEMPORARY FUNCTION add_bias AS 'hivemall.ftvec.AddBiasUDF';
CREATE TEMPORARY FUNCTION sortByFeature AS
'hivemall.ftvec.SortByFeatureUDF';
CREATE TEMPORARY FUNCTION sort_by_feature AS
'hivemall.ftvec.SortByFeatureUDF';
CREATE TEMPORARY FUNCTION extract_feature AS
'hivemall.ftvec.ExtractFeatureUDF';
CREATE TEMPORARY FUNCTION extract_feature AS
'hivemall.ftvec.ExtractFeatureUDF';
CREATE TEMPORARY FUNCTION extract_weight AS
'hivemall.ftvec.ExtractWeightUDF';
CREATE TEMPORARY FUNCTION add_feature_index AS
'hivemall.ftvec.AddFeatureIndexUDF';
CREATE TEMPORARY FUNCTION feature AS 'hivemall.ftvec.FeatureUDF';
CREATE TEMPORARY FUNCTION feature_index AS
'hivemall.ftvec.FeatureIndexUDF';
CREATE TEMPORARY FUNCTION tf AS 'hivemall.ftvec.text.TermFrequencyUDAF';
CREATE TEMPORARY FUNCTION train_logregr AS
'hivemall.regression.LogressUDTF';
CREATE TEMPORARY FUNCTION train_pa1_regr AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION train_pa1a_regr AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION train_pa2_regr AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION train_pa2a_regr AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION train_arow_regr AS
'hivemall.regression.AROWRegressionUDTF';
CREATE TEMPORARY FUNCTION train_arowe_regr AS
'hivemall.regression.AROWRegressionUDTF';
CREATE TEMPORARY FUNCTION train_arowe2_regr AS
'hivemall.regression.AROWRegressionUDTF';
CREATE TEMPORARY FUNCTION train_adagrad_regr AS
'hivemall.regression.AdaGradUDTF';
CREATE TEMPORARY FUNCTION train_adadelta_regr AS
'hivemall.regression.AdaDeltaUDTF';
CREATE TEMPORARY FUNCTION train_adagrad AS
'hivemall.regression.AdaGradUDTF';
CREATE TEMPORARY FUNCTION train_adagrad AS
'hivemall.regression.AdaGradUDTF';
CREATE TEMPORARY FUNCTION train_adadelta AS
'hivemall.regression.AdaDeltaUDTF';
CREATE TEMPORARY FUNCTION logress AS 'hivemall.regression.LogressUDTF';
CREATE TEMPORARY FUNCTION pa1_regress AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION pa1a_regress AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION pa2_regress AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION pa2a_regress AS
'hivemall.regression.PassiveAggressiveRegressionUDTF';
CREATE TEMPORARY FUNCTION arow_regress AS
'hivemall.regression.AROWRegressionUDTF';
CREATE TEMPORARY FUNCTION arowe_regress AS
'hivemall.regression.AROWRegressionUDTF';
CREATE TEMPORARY FUNCTION arowe2_regress AS
'hivemall.regression.AROWRegressionUDTF';
CREATE TEMPORARY FUNCTION adagrad AS 'hivemall.regression.AdaGradUDTF';
CREATE TEMPORARY FUNCTION adadelta AS 'hivemall.regression.AdaDeltaUDTF';
CREATE TEMPORARY FUNCTION float_array AS
'hivemall.tools.array.AllocFloatArrayUDF';
CREATE TEMPORARY FUNCTION array_remove AS
'hivemall.tools.array.ArrayRemoveUDF';
CREATE TEMPORARY FUNCTION sort_and_uniq_array AS
'hivemall.tools.array.SortAndUniqArrayUDF';
CREATE TEMPORARY FUNCTION subarray_endwith AS
'hivemall.tools.array.SubarrayEndWithUDF';
CREATE TEMPORARY FUNCTION subarray_startwith AS
'hivemall.tools.array.SubarrayStartWithUDF';
CREATE TEMPORARY FUNCTION collect_all AS
CREATE TEMPORARY FUNCTION collect_all AS
'hivemall.tools.array.CollectAllUDAF';
CREATE TEMPORARY FUNCTION concat_array AS
'hivemall.tools.array.ConcatArrayUDF';
CREATE TEMPORARY FUNCTION subarray AS 'hivemall.tools.array.SubarrayUDF';
CREATE TEMPORARY FUNCTION array_avg AS
'hivemall.tools.array.ArrayAvgGenericUDAF';
CREATE TEMPORARY FUNCTION array_sum AS
'hivemall.tools.array.ArraySumUDAF';
CREATE TEMPORARY FUNCTION to_string_array AS
'hivemall.tools.array.ToStringArrayUDF';
CREATE TEMPORARY FUNCTION map_get_sum AS
'hivemall.tools.map.MapGetSumUDF';
CREATE TEMPORARY FUNCTION map_tail_n AS 'hivemall.tools.map.MapTailNUDF';
CREATE TEMPORARY FUNCTION to_map AS 'hivemall.tools.map.UDAFToMap';
CREATE TEMPORARY FUNCTION to_ordered_map AS
'hivemall.tools.map.UDAFToOrderedMap';
CREATE TEMPORARY FUNCTION sigmoid AS
'hivemall.tools.math.SigmoidGenericUDF';
CREATE TEMPORARY FUNCTION taskid AS 'hivemall.tools.mapred.TaskIdUDF';
CREATE TEMPORARY FUNCTION jobid AS 'hivemall.tools.mapred.JobIdUDF';
CREATE TEMPORARY FUNCTION rowid AS 'hivemall.tools.mapred.RowIdUDF';
CREATE TEMPORARY FUNCTION generate_series AS
'hivemall.tools.GenerateSeriesUDTF';
CREATE TEMPORARY FUNCTION convert_label AS
'hivemall.tools.ConvertLabelUDF';
CREATE TEMPORARY FUNCTION x_rank AS 'hivemall.tools.RankSequenceUDF';
CREATE TEMPORARY FUNCTION each_top_k AS 'hivemall.tools.EachTopKUDTF';
CREATE TEMPORARY FUNCTION tokenize AS 'hivemall.tools.text.TokenizeUDF';
CREATE TEMPORARY FUNCTION is_stopword AS
'hivemall.tools.text.StopwordUDF';
CREATE TEMPORARY FUNCTION split_words AS
CREATE TEMPORARY FUNCTION split_words AS
'hivemall.tools.text.SplitWordsUDF';
CREATE TEMPORARY FUNCTION normalize_unicode AS
'hivemall.tools.text.NormalizeUnicodeUDF';
CREATE TEMPORARY FUNCTION lr_datagen AS
'hivemall.dataset.LogisticRegressionDataGeneratorUDTF';
CREATE TEMPORARY FUNCTION f1score AS 'hivemall.evaluation.FMeasureUDAF';
CREATE TEMPORARY FUNCTION mae AS
'hivemall.evaluation.MeanAbsoluteErrorUDAF';
CREATE TEMPORARY FUNCTION mse AS
'hivemall.evaluation.MeanSquaredErrorUDAF';
CREATE TEMPORARY FUNCTION rmse AS
'hivemall.evaluation.RootMeanSquaredErrorUDAF';
CREATE TEMPORARY FUNCTION mf_predict AS 'hivemall.mf.MFPredictionUDF';
CREATE TEMPORARY FUNCTION train_mf_sgd AS
'hivemall.mf.MatrixFactorizationSGDUDTF';
CREATE TEMPORARY FUNCTION train_mf_adagrad AS
'hivemall.mf.MatrixFactorizationAdaGradUDTF';
CREATE TEMPORARY FUNCTION fm_predict AS
'hivemall.fm.FMPredictGenericUDAF';
CREATE TEMPORARY FUNCTION train_fm AS
'hivemall.fm.FactorizationMachineUDTF';
CREATE TEMPORARY FUNCTION train_randomforest_classifier AS
'hivemall.smile.classification.RandomForestClassifierUDTF';
CREATE TEMPORARY FUNCTION train_rf_classifier AS
'hivemall.smile.classification.RandomForestClassifierUDTF';
CREATE TEMPORARY FUNCTION train_randomforest_regr AS
'hivemall.smile.regression.RandomForestRegressionUDTF';
CREATE TEMPORARY FUNCTION train_rf_regr AS
'hivemall.smile.regression.RandomForestRegressionUDTF';
CREATE TEMPORARY FUNCTION tree_predict AS
'hivemall.smile.tools.TreePredictByStackMachineUDF';
CREATE TEMPORARY FUNCTION tree_predict AS
'hivemall.smile.tools.TreePredictByStackMachineUDF';
CREATE TEMPORARY FUNCTION vm_tree_predict AS
'hivemall.smile.tools.TreePredictByStackMachineUDF';
CREATE TEMPORARY FUNCTION rf_ensemble AS
'hivemall.smile.tools.RandomForestEnsembleUDAF';
CREATE TEMPORARY FUNCTION train_gradient_boosting_classifier AS
'hivemall.smile.classification.GradientTreeBoostingClassifierUDTF';
CREATE TEMPORARY FUNCTION guess_attribute_types AS
'hivemall.smile.tools.GuessAttributesUDF';
CREATE TEMPORARY FUNCTION tokenize_ja AS
'hivemall.nlp.tokenizer.KuromojiUDF';
CREATE TEMPORARY MACRO max2(x DOUBLE, y DOUBLE) if(x>y,x,y);
CREATE TEMPORARY MACRO min2(x DOUBLE, y DOUBLE) if(x<y,x,y);
CREATE TEMPORARY MACRO rand_gid(k INT) floor(rand()*k);
CREATE TEMPORARY MACRO rand_gid2(k INT, seed INT) floor(rand(seed)*k);
CREATE TEMPORARY MACRO idf(df_t DOUBLE, n_docs DOUBLE) log(10, n_docs /
max2(1,df_t)) + 1.0;
CREATE TEMPORARY MACRO tfidf(tf FLOAT, df_t DOUBLE, n_docs DOUBLE) tf *
(log(10, n_docs / max2(1,df_t)) + 1.0);
SELECT time, COUNT(1) AS cnt FROM tbl1
WHERE TD_TIME_RANGE(time, '2015-12-11', '2015-12-12', 'JST');
Do you still love
Java / SQL ?
PQ written in Ruby
• Building jobs/parameters is so complex!
• using data from many configurations (YAML, JSON),
internal APIs and RDBMSs
• with many ext syntaxes/rules to tune performance,
override configurations for tests, ...
• Ruby empower to write fat/complex worker code
• Testing!
• Unit tests using Rspec
• System tests (executing real queries/jobs) using
Rspec
For Further improvement
about workers
• More performance for more customers and less
costs
• More scalability for many other kind jobs
• Better and well-controlled tests (indented here
documents!)
"Done is better than Perfect."
PerfectQueue
DoneQueue
?
We'll improve our code step by step,
with improvements of ruby and developer
community <3
Thanks!

More Related Content

What's hot

Presto in Treasure Data (presented at db tech showcase Sapporo 2015)
Presto in Treasure Data (presented at db tech showcase Sapporo 2015)Presto in Treasure Data (presented at db tech showcase Sapporo 2015)
Presto in Treasure Data (presented at db tech showcase Sapporo 2015)Mitsunori Komatsu
 
Open Source Software, Distributed Systems, Database as a Cloud Service
Open Source Software, Distributed Systems, Database as a Cloud ServiceOpen Source Software, Distributed Systems, Database as a Cloud Service
Open Source Software, Distributed Systems, Database as a Cloud ServiceSATOSHI TAGOMORI
 
Presto At Treasure Data
Presto At Treasure DataPresto At Treasure Data
Presto At Treasure DataTaro L. Saito
 
Automating Workflows for Analytics Pipelines
Automating Workflows for Analytics PipelinesAutomating Workflows for Analytics Pipelines
Automating Workflows for Analytics PipelinesSadayuki Furuhashi
 
How To Write Middleware In Ruby
How To Write Middleware In RubyHow To Write Middleware In Ruby
How To Write Middleware In RubySATOSHI TAGOMORI
 
Digdagによる大規模データ処理の自動化とエラー処理
Digdagによる大規模データ処理の自動化とエラー処理Digdagによる大規模データ処理の自動化とエラー処理
Digdagによる大規模データ処理の自動化とエラー処理Sadayuki Furuhashi
 
Fluentd Overview, Now and Then
Fluentd Overview, Now and ThenFluentd Overview, Now and Then
Fluentd Overview, Now and ThenSATOSHI TAGOMORI
 
Habits of Effective Sqoop Users
Habits of Effective Sqoop UsersHabits of Effective Sqoop Users
Habits of Effective Sqoop UsersKathleen Ting
 
London devops logging
London devops loggingLondon devops logging
London devops loggingTomas Doran
 
User Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDBUser Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDBKai Sasaki
 
Debugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauDebugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauSpark Summit
 
Using Morphlines for On-the-Fly ETL
Using Morphlines for On-the-Fly ETLUsing Morphlines for On-the-Fly ETL
Using Morphlines for On-the-Fly ETLCloudera, Inc.
 
Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)
Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)
Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)Maarten Balliauw
 
Solr for Indexing and Searching Logs
Solr for Indexing and Searching LogsSolr for Indexing and Searching Logs
Solr for Indexing and Searching LogsSematext Group, Inc.
 
Building a near real time search engine & analytics for logs using solr
Building a near real time search engine & analytics for logs using solrBuilding a near real time search engine & analytics for logs using solr
Building a near real time search engine & analytics for logs using solrlucenerevolution
 
Hadoop, Hive, Spark and Object Stores
Hadoop, Hive, Spark and Object StoresHadoop, Hive, Spark and Object Stores
Hadoop, Hive, Spark and Object StoresSteve Loughran
 

What's hot (20)

Presto in Treasure Data (presented at db tech showcase Sapporo 2015)
Presto in Treasure Data (presented at db tech showcase Sapporo 2015)Presto in Treasure Data (presented at db tech showcase Sapporo 2015)
Presto in Treasure Data (presented at db tech showcase Sapporo 2015)
 
Open Source Software, Distributed Systems, Database as a Cloud Service
Open Source Software, Distributed Systems, Database as a Cloud ServiceOpen Source Software, Distributed Systems, Database as a Cloud Service
Open Source Software, Distributed Systems, Database as a Cloud Service
 
Presto At Treasure Data
Presto At Treasure DataPresto At Treasure Data
Presto At Treasure Data
 
Automating Workflows for Analytics Pipelines
Automating Workflows for Analytics PipelinesAutomating Workflows for Analytics Pipelines
Automating Workflows for Analytics Pipelines
 
How To Write Middleware In Ruby
How To Write Middleware In RubyHow To Write Middleware In Ruby
How To Write Middleware In Ruby
 
Making KVS 10x Scalable
Making KVS 10x ScalableMaking KVS 10x Scalable
Making KVS 10x Scalable
 
Handling not so big data
Handling not so big dataHandling not so big data
Handling not so big data
 
Digdagによる大規模データ処理の自動化とエラー処理
Digdagによる大規模データ処理の自動化とエラー処理Digdagによる大規模データ処理の自動化とエラー処理
Digdagによる大規模データ処理の自動化とエラー処理
 
Fluentd Overview, Now and Then
Fluentd Overview, Now and ThenFluentd Overview, Now and Then
Fluentd Overview, Now and Then
 
Debugging Apache Spark
Debugging Apache SparkDebugging Apache Spark
Debugging Apache Spark
 
Habits of Effective Sqoop Users
Habits of Effective Sqoop UsersHabits of Effective Sqoop Users
Habits of Effective Sqoop Users
 
London devops logging
London devops loggingLondon devops logging
London devops logging
 
User Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDBUser Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDB
 
Debugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauDebugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden Karau
 
Tuning Solr & Pipeline for Logs
Tuning Solr & Pipeline for LogsTuning Solr & Pipeline for Logs
Tuning Solr & Pipeline for Logs
 
Using Morphlines for On-the-Fly ETL
Using Morphlines for On-the-Fly ETLUsing Morphlines for On-the-Fly ETL
Using Morphlines for On-the-Fly ETL
 
Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)
Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)
Get more than a cache back! The Microsoft Azure Redis Cache (NDC Oslo)
 
Solr for Indexing and Searching Logs
Solr for Indexing and Searching LogsSolr for Indexing and Searching Logs
Solr for Indexing and Searching Logs
 
Building a near real time search engine & analytics for logs using solr
Building a near real time search engine & analytics for logs using solrBuilding a near real time search engine & analytics for logs using solr
Building a near real time search engine & analytics for logs using solr
 
Hadoop, Hive, Spark and Object Stores
Hadoop, Hive, Spark and Object StoresHadoop, Hive, Spark and Object Stores
Hadoop, Hive, Spark and Object Stores
 

Viewers also liked

How to create/improve OSS product and its community (revised)
How to create/improve OSS product and its community (revised)How to create/improve OSS product and its community (revised)
How to create/improve OSS product and its community (revised)SATOSHI TAGOMORI
 
Data Analytics Service Company and Its Ruby Usage
Data Analytics Service Company and Its Ruby UsageData Analytics Service Company and Its Ruby Usage
Data Analytics Service Company and Its Ruby UsageSATOSHI TAGOMORI
 
RubyKaigi2015 making robots-with-mruby
RubyKaigi2015 making robots-with-mrubyRubyKaigi2015 making robots-with-mruby
RubyKaigi2015 making robots-with-mrubyyamanekko
 
The OMR GC talk - Ruby Kaigi 2015
The OMR GC talk - Ruby Kaigi 2015The OMR GC talk - Ruby Kaigi 2015
The OMR GC talk - Ruby Kaigi 2015craig lehmann
 
"fireap" - fast task runner on consul
"fireap" - fast task runner on consul"fireap" - fast task runner on consul
"fireap" - fast task runner on consulIKEDA Kiyoshi
 
grifork - fast propagative task runner -
grifork - fast propagative task runner -grifork - fast propagative task runner -
grifork - fast propagative task runner -IKEDA Kiyoshi
 
Introduction to poloxy - proxy for alerting
Introduction to poloxy - proxy for alertingIntroduction to poloxy - proxy for alerting
Introduction to poloxy - proxy for alertingIKEDA Kiyoshi
 
Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)
Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)
Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)ngotogenome
 
Distributed Logging Architecture in Container Era
Distributed Logging Architecture in Container EraDistributed Logging Architecture in Container Era
Distributed Logging Architecture in Container EraSATOSHI TAGOMORI
 
The worst Ruby codes I’ve seen in my life - RubyKaigi 2015
The worst Ruby codes I’ve seen in my life - RubyKaigi 2015The worst Ruby codes I’ve seen in my life - RubyKaigi 2015
The worst Ruby codes I’ve seen in my life - RubyKaigi 2015Fernando Hamasaki de Amorim
 
Introduction to Meteor & React
Introduction to Meteor & ReactIntroduction to Meteor & React
Introduction to Meteor & ReactMax Li
 
つくること = 生きること : パターン・ランゲージによる創造の支援
つくること = 生きること : パターン・ランゲージによる創造の支援 つくること = 生きること : パターン・ランゲージによる創造の支援
つくること = 生きること : パターン・ランゲージによる創造の支援 Takashi Iba
 
FizzBuzzではじめるテスト
FizzBuzzではじめるテストFizzBuzzではじめるテスト
FizzBuzzではじめるテストMasashi Shinbara
 
Fluentd v0.14 Plugin API Details
Fluentd v0.14 Plugin API DetailsFluentd v0.14 Plugin API Details
Fluentd v0.14 Plugin API DetailsSATOSHI TAGOMORI
 
Fighting API Compatibility On Fluentd Using "Black Magic"
Fighting API Compatibility On Fluentd Using "Black Magic"Fighting API Compatibility On Fluentd Using "Black Magic"
Fighting API Compatibility On Fluentd Using "Black Magic"SATOSHI TAGOMORI
 
せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜
せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜
せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜Junichi Noda
 
Practical Testing of Ruby Core
Practical Testing of Ruby CorePractical Testing of Ruby Core
Practical Testing of Ruby CoreHiroshi SHIBATA
 
Experiments in Sharing Java VM Technology with CRuby
Experiments in Sharing Java VM Technology with CRubyExperiments in Sharing Java VM Technology with CRuby
Experiments in Sharing Java VM Technology with CRubyMatthew Gaudet
 
Redis勉強会資料(2015/06 update)
Redis勉強会資料(2015/06 update)Redis勉強会資料(2015/06 update)
Redis勉強会資料(2015/06 update)Yuji Otani
 

Viewers also liked (20)

How to create/improve OSS product and its community (revised)
How to create/improve OSS product and its community (revised)How to create/improve OSS product and its community (revised)
How to create/improve OSS product and its community (revised)
 
Data Analytics Service Company and Its Ruby Usage
Data Analytics Service Company and Its Ruby UsageData Analytics Service Company and Its Ruby Usage
Data Analytics Service Company and Its Ruby Usage
 
RubyKaigi2015 making robots-with-mruby
RubyKaigi2015 making robots-with-mrubyRubyKaigi2015 making robots-with-mruby
RubyKaigi2015 making robots-with-mruby
 
The OMR GC talk - Ruby Kaigi 2015
The OMR GC talk - Ruby Kaigi 2015The OMR GC talk - Ruby Kaigi 2015
The OMR GC talk - Ruby Kaigi 2015
 
Ruby meets Go
Ruby meets GoRuby meets Go
Ruby meets Go
 
"fireap" - fast task runner on consul
"fireap" - fast task runner on consul"fireap" - fast task runner on consul
"fireap" - fast task runner on consul
 
grifork - fast propagative task runner -
grifork - fast propagative task runner -grifork - fast propagative task runner -
grifork - fast propagative task runner -
 
Introduction to poloxy - proxy for alerting
Introduction to poloxy - proxy for alertingIntroduction to poloxy - proxy for alerting
Introduction to poloxy - proxy for alerting
 
Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)
Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)
Running Ruby on Solaris (RubyKaigi 2015, 12/Dec/2015)
 
Distributed Logging Architecture in Container Era
Distributed Logging Architecture in Container EraDistributed Logging Architecture in Container Era
Distributed Logging Architecture in Container Era
 
The worst Ruby codes I’ve seen in my life - RubyKaigi 2015
The worst Ruby codes I’ve seen in my life - RubyKaigi 2015The worst Ruby codes I’ve seen in my life - RubyKaigi 2015
The worst Ruby codes I’ve seen in my life - RubyKaigi 2015
 
Introduction to Meteor & React
Introduction to Meteor & ReactIntroduction to Meteor & React
Introduction to Meteor & React
 
つくること = 生きること : パターン・ランゲージによる創造の支援
つくること = 生きること : パターン・ランゲージによる創造の支援 つくること = 生きること : パターン・ランゲージによる創造の支援
つくること = 生きること : パターン・ランゲージによる創造の支援
 
FizzBuzzではじめるテスト
FizzBuzzではじめるテストFizzBuzzではじめるテスト
FizzBuzzではじめるテスト
 
Fluentd v0.14 Plugin API Details
Fluentd v0.14 Plugin API DetailsFluentd v0.14 Plugin API Details
Fluentd v0.14 Plugin API Details
 
Fighting API Compatibility On Fluentd Using "Black Magic"
Fighting API Compatibility On Fluentd Using "Black Magic"Fighting API Compatibility On Fluentd Using "Black Magic"
Fighting API Compatibility On Fluentd Using "Black Magic"
 
せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜
せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜
せいまち〜聖地探訪に出会いを求めるのは間違っているだろうか〜
 
Practical Testing of Ruby Core
Practical Testing of Ruby CorePractical Testing of Ruby Core
Practical Testing of Ruby Core
 
Experiments in Sharing Java VM Technology with CRuby
Experiments in Sharing Java VM Technology with CRubyExperiments in Sharing Java VM Technology with CRuby
Experiments in Sharing Java VM Technology with CRuby
 
Redis勉強会資料(2015/06 update)
Redis勉強会資料(2015/06 update)Redis勉強会資料(2015/06 update)
Redis勉強会資料(2015/06 update)
 

Similar to Data Analytics Service Company and Its Ruby Usage

ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupRafal Kwasny
 
StackMate - CloudFormation for CloudStack
StackMate - CloudFormation for CloudStackStackMate - CloudFormation for CloudStack
StackMate - CloudFormation for CloudStackChiradeep Vittal
 
Buildingsocialanalyticstoolwithmongodb
BuildingsocialanalyticstoolwithmongodbBuildingsocialanalyticstoolwithmongodb
BuildingsocialanalyticstoolwithmongodbMongoDB APAC
 
Hive dirty/beautiful hacks in TD
Hive dirty/beautiful hacks in TDHive dirty/beautiful hacks in TD
Hive dirty/beautiful hacks in TDSATOSHI TAGOMORI
 
Introduction to real time big data with Apache Spark
Introduction to real time big data with Apache SparkIntroduction to real time big data with Apache Spark
Introduction to real time big data with Apache SparkTaras Matyashovsky
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedwhoschek
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoopclairvoyantllc
 
SF Big Analytics meetup : Hoodie From Uber
SF Big Analytics meetup : Hoodie  From UberSF Big Analytics meetup : Hoodie  From Uber
SF Big Analytics meetup : Hoodie From UberChester Chen
 
20151010 my sq-landjavav2a
20151010 my sq-landjavav2a20151010 my sq-landjavav2a
20151010 my sq-landjavav2aIvan Ma
 
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...Chester Chen
 
SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15SnappyData
 
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael HausenblasBerlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael HausenblasMapR Technologies
 
Creating PostgreSQL-as-a-Service at Scale
Creating PostgreSQL-as-a-Service at ScaleCreating PostgreSQL-as-a-Service at Scale
Creating PostgreSQL-as-a-Service at ScaleSean Chittenden
 
20160307 apex connects_jira
20160307 apex connects_jira20160307 apex connects_jira
20160307 apex connects_jiraMT AG
 
Simplifying Apache Cascading
Simplifying Apache CascadingSimplifying Apache Cascading
Simplifying Apache CascadingMing Yuan
 
Rhebok, High Performance Rack Handler / Rubykaigi 2015
Rhebok, High Performance Rack Handler / Rubykaigi 2015Rhebok, High Performance Rack Handler / Rubykaigi 2015
Rhebok, High Performance Rack Handler / Rubykaigi 2015Masahiro Nagano
 
비동기 회고 발표자료
비동기 회고 발표자료비동기 회고 발표자료
비동기 회고 발표자료Benjamin Kim
 
Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
 Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov... Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...Databricks
 
Building Scalable Websites with Perl
Building Scalable Websites with PerlBuilding Scalable Websites with Perl
Building Scalable Websites with PerlPerrin Harkins
 

Similar to Data Analytics Service Company and Its Ruby Usage (20)

ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 
StackMate - CloudFormation for CloudStack
StackMate - CloudFormation for CloudStackStackMate - CloudFormation for CloudStack
StackMate - CloudFormation for CloudStack
 
Buildingsocialanalyticstoolwithmongodb
BuildingsocialanalyticstoolwithmongodbBuildingsocialanalyticstoolwithmongodb
Buildingsocialanalyticstoolwithmongodb
 
Hive dirty/beautiful hacks in TD
Hive dirty/beautiful hacks in TDHive dirty/beautiful hacks in TD
Hive dirty/beautiful hacks in TD
 
Introduction to real time big data with Apache Spark
Introduction to real time big data with Apache SparkIntroduction to real time big data with Apache Spark
Introduction to real time big data with Apache Spark
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmed
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoop
 
SF Big Analytics meetup : Hoodie From Uber
SF Big Analytics meetup : Hoodie  From UberSF Big Analytics meetup : Hoodie  From Uber
SF Big Analytics meetup : Hoodie From Uber
 
20151010 my sq-landjavav2a
20151010 my sq-landjavav2a20151010 my sq-landjavav2a
20151010 my sq-landjavav2a
 
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
 
SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15
 
Nike tech talk.2
Nike tech talk.2Nike tech talk.2
Nike tech talk.2
 
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael HausenblasBerlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
 
Creating PostgreSQL-as-a-Service at Scale
Creating PostgreSQL-as-a-Service at ScaleCreating PostgreSQL-as-a-Service at Scale
Creating PostgreSQL-as-a-Service at Scale
 
20160307 apex connects_jira
20160307 apex connects_jira20160307 apex connects_jira
20160307 apex connects_jira
 
Simplifying Apache Cascading
Simplifying Apache CascadingSimplifying Apache Cascading
Simplifying Apache Cascading
 
Rhebok, High Performance Rack Handler / Rubykaigi 2015
Rhebok, High Performance Rack Handler / Rubykaigi 2015Rhebok, High Performance Rack Handler / Rubykaigi 2015
Rhebok, High Performance Rack Handler / Rubykaigi 2015
 
비동기 회고 발표자료
비동기 회고 발표자료비동기 회고 발표자료
비동기 회고 발표자료
 
Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
 Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov... Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
 
Building Scalable Websites with Perl
Building Scalable Websites with PerlBuilding Scalable Websites with Perl
Building Scalable Websites with Perl
 

More from SATOSHI TAGOMORI

Ractor's speed is not light-speed
Ractor's speed is not light-speedRactor's speed is not light-speed
Ractor's speed is not light-speedSATOSHI TAGOMORI
 
Good Things and Hard Things of SaaS Development/Operations
Good Things and Hard Things of SaaS Development/OperationsGood Things and Hard Things of SaaS Development/Operations
Good Things and Hard Things of SaaS Development/OperationsSATOSHI TAGOMORI
 
Invitation to the dark side of Ruby
Invitation to the dark side of RubyInvitation to the dark side of Ruby
Invitation to the dark side of RubySATOSHI TAGOMORI
 
Hijacking Ruby Syntax in Ruby (RubyConf 2018)
Hijacking Ruby Syntax in Ruby (RubyConf 2018)Hijacking Ruby Syntax in Ruby (RubyConf 2018)
Hijacking Ruby Syntax in Ruby (RubyConf 2018)SATOSHI TAGOMORI
 
Make Your Ruby Script Confusing
Make Your Ruby Script ConfusingMake Your Ruby Script Confusing
Make Your Ruby Script ConfusingSATOSHI TAGOMORI
 
Hijacking Ruby Syntax in Ruby
Hijacking Ruby Syntax in RubyHijacking Ruby Syntax in Ruby
Hijacking Ruby Syntax in RubySATOSHI TAGOMORI
 
Lock, Concurrency and Throughput of Exclusive Operations
Lock, Concurrency and Throughput of Exclusive OperationsLock, Concurrency and Throughput of Exclusive Operations
Lock, Concurrency and Throughput of Exclusive OperationsSATOSHI TAGOMORI
 
Data Processing and Ruby in the World
Data Processing and Ruby in the WorldData Processing and Ruby in the World
Data Processing and Ruby in the WorldSATOSHI TAGOMORI
 
Planet-scale Data Ingestion Pipeline: Bigdam
Planet-scale Data Ingestion Pipeline: BigdamPlanet-scale Data Ingestion Pipeline: Bigdam
Planet-scale Data Ingestion Pipeline: BigdamSATOSHI TAGOMORI
 
Technologies, Data Analytics Service and Enterprise Business
Technologies, Data Analytics Service and Enterprise BusinessTechnologies, Data Analytics Service and Enterprise Business
Technologies, Data Analytics Service and Enterprise BusinessSATOSHI TAGOMORI
 
Ruby and Distributed Storage Systems
Ruby and Distributed Storage SystemsRuby and Distributed Storage Systems
Ruby and Distributed Storage SystemsSATOSHI TAGOMORI
 
Perfect Norikra 2nd Season
Perfect Norikra 2nd SeasonPerfect Norikra 2nd Season
Perfect Norikra 2nd SeasonSATOSHI TAGOMORI
 
To Have Own Data Analytics Platform, Or NOT To
To Have Own Data Analytics Platform, Or NOT ToTo Have Own Data Analytics Platform, Or NOT To
To Have Own Data Analytics Platform, Or NOT ToSATOSHI TAGOMORI
 
The Patterns of Distributed Logging and Containers
The Patterns of Distributed Logging and ContainersThe Patterns of Distributed Logging and Containers
The Patterns of Distributed Logging and ContainersSATOSHI TAGOMORI
 
Modern Black Mages Fighting in the Real World
Modern Black Mages Fighting in the Real WorldModern Black Mages Fighting in the Real World
Modern Black Mages Fighting in the Real WorldSATOSHI TAGOMORI
 
Tale of ISUCON and Its Bench Tools
Tale of ISUCON and Its Bench ToolsTale of ISUCON and Its Bench Tools
Tale of ISUCON and Its Bench ToolsSATOSHI TAGOMORI
 
Data-Driven Development Era and Its Technologies
Data-Driven Development Era and Its TechnologiesData-Driven Development Era and Its Technologies
Data-Driven Development Era and Its TechnologiesSATOSHI TAGOMORI
 
Engineer as a Leading Role
Engineer as a Leading RoleEngineer as a Leading Role
Engineer as a Leading RoleSATOSHI TAGOMORI
 

More from SATOSHI TAGOMORI (20)

Ractor's speed is not light-speed
Ractor's speed is not light-speedRactor's speed is not light-speed
Ractor's speed is not light-speed
 
Good Things and Hard Things of SaaS Development/Operations
Good Things and Hard Things of SaaS Development/OperationsGood Things and Hard Things of SaaS Development/Operations
Good Things and Hard Things of SaaS Development/Operations
 
Maccro Strikes Back
Maccro Strikes BackMaccro Strikes Back
Maccro Strikes Back
 
Invitation to the dark side of Ruby
Invitation to the dark side of RubyInvitation to the dark side of Ruby
Invitation to the dark side of Ruby
 
Hijacking Ruby Syntax in Ruby (RubyConf 2018)
Hijacking Ruby Syntax in Ruby (RubyConf 2018)Hijacking Ruby Syntax in Ruby (RubyConf 2018)
Hijacking Ruby Syntax in Ruby (RubyConf 2018)
 
Make Your Ruby Script Confusing
Make Your Ruby Script ConfusingMake Your Ruby Script Confusing
Make Your Ruby Script Confusing
 
Hijacking Ruby Syntax in Ruby
Hijacking Ruby Syntax in RubyHijacking Ruby Syntax in Ruby
Hijacking Ruby Syntax in Ruby
 
Lock, Concurrency and Throughput of Exclusive Operations
Lock, Concurrency and Throughput of Exclusive OperationsLock, Concurrency and Throughput of Exclusive Operations
Lock, Concurrency and Throughput of Exclusive Operations
 
Data Processing and Ruby in the World
Data Processing and Ruby in the WorldData Processing and Ruby in the World
Data Processing and Ruby in the World
 
Planet-scale Data Ingestion Pipeline: Bigdam
Planet-scale Data Ingestion Pipeline: BigdamPlanet-scale Data Ingestion Pipeline: Bigdam
Planet-scale Data Ingestion Pipeline: Bigdam
 
Technologies, Data Analytics Service and Enterprise Business
Technologies, Data Analytics Service and Enterprise BusinessTechnologies, Data Analytics Service and Enterprise Business
Technologies, Data Analytics Service and Enterprise Business
 
Ruby and Distributed Storage Systems
Ruby and Distributed Storage SystemsRuby and Distributed Storage Systems
Ruby and Distributed Storage Systems
 
Perfect Norikra 2nd Season
Perfect Norikra 2nd SeasonPerfect Norikra 2nd Season
Perfect Norikra 2nd Season
 
Fluentd 101
Fluentd 101Fluentd 101
Fluentd 101
 
To Have Own Data Analytics Platform, Or NOT To
To Have Own Data Analytics Platform, Or NOT ToTo Have Own Data Analytics Platform, Or NOT To
To Have Own Data Analytics Platform, Or NOT To
 
The Patterns of Distributed Logging and Containers
The Patterns of Distributed Logging and ContainersThe Patterns of Distributed Logging and Containers
The Patterns of Distributed Logging and Containers
 
Modern Black Mages Fighting in the Real World
Modern Black Mages Fighting in the Real WorldModern Black Mages Fighting in the Real World
Modern Black Mages Fighting in the Real World
 
Tale of ISUCON and Its Bench Tools
Tale of ISUCON and Its Bench ToolsTale of ISUCON and Its Bench Tools
Tale of ISUCON and Its Bench Tools
 
Data-Driven Development Era and Its Technologies
Data-Driven Development Era and Its TechnologiesData-Driven Development Era and Its Technologies
Data-Driven Development Era and Its Technologies
 
Engineer as a Leading Role
Engineer as a Leading RoleEngineer as a Leading Role
Engineer as a Leading Role
 

Recently uploaded

Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Hr365.us smith
 
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样umasea
 
Cloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEECloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEEVICTOR MAESTRE RAMIREZ
 
Introduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfIntroduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfFerryKemperman
 
MYjobs Presentation Django-based project
MYjobs Presentation Django-based projectMYjobs Presentation Django-based project
MYjobs Presentation Django-based projectAnoyGreter
 
Best Web Development Agency- Idiosys USA.pdf
Best Web Development Agency- Idiosys USA.pdfBest Web Development Agency- Idiosys USA.pdf
Best Web Development Agency- Idiosys USA.pdfIdiosysTechnologies1
 
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEBATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEOrtus Solutions, Corp
 
How to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfHow to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfLivetecs LLC
 
Unveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML DiagramsUnveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML DiagramsAhmed Mohamed
 
React Server Component in Next.js by Hanief Utama
React Server Component in Next.js by Hanief UtamaReact Server Component in Next.js by Hanief Utama
React Server Component in Next.js by Hanief UtamaHanief Utama
 
EY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityEY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityNeo4j
 
What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...Technogeeks
 
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdfGOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdfAlina Yurenko
 
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...OnePlan Solutions
 
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company OdishaBalasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odishasmiwainfosol
 
Cyber security and its impact on E commerce
Cyber security and its impact on E commerceCyber security and its impact on E commerce
Cyber security and its impact on E commercemanigoyal112
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...OnePlan Solutions
 
Folding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesFolding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesPhilip Schwarz
 
Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...Velvetech LLC
 
Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024
Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024
Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024StefanoLambiase
 

Recently uploaded (20)

Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)
 
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
 
Cloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEECloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEE
 
Introduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfIntroduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdf
 
MYjobs Presentation Django-based project
MYjobs Presentation Django-based projectMYjobs Presentation Django-based project
MYjobs Presentation Django-based project
 
Best Web Development Agency- Idiosys USA.pdf
Best Web Development Agency- Idiosys USA.pdfBest Web Development Agency- Idiosys USA.pdf
Best Web Development Agency- Idiosys USA.pdf
 
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEBATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
 
How to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfHow to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdf
 
Unveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML DiagramsUnveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML Diagrams
 
React Server Component in Next.js by Hanief Utama
React Server Component in Next.js by Hanief UtamaReact Server Component in Next.js by Hanief Utama
React Server Component in Next.js by Hanief Utama
 
EY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityEY_Graph Database Powered Sustainability
EY_Graph Database Powered Sustainability
 
What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...
 
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdfGOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
 
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
 
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company OdishaBalasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
 
Cyber security and its impact on E commerce
Cyber security and its impact on E commerceCyber security and its impact on E commerce
Cyber security and its impact on E commerce
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
 
Folding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesFolding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a series
 
Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...
 
Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024
Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024
Dealing with Cultural Dispersion — Stefano Lambiase — ICSE-SEIS 2024
 

Data Analytics Service Company and Its Ruby Usage

  • 1. Data Analytics Service Company and Its Ruby Usage RubyKaigi 2015 (Dec 12, 2015) Satoshi Tagomori (@tagomoris)
  • 2. Satoshi "Moris" Tagomori (@tagomoris) Fluentd, MessagePack-Ruby, Norikra, ... Treasure Data, Inc.
  • 3.
  • 5. http://www.fluentd.org/ Fluentd Unified Logging Layer For Stream Data Written in CRuby http://www.slideshare.net/treasure-data/the-basics-of-fluentd-35681111
  • 6. Bulk Data Loader High Throughput&Reliability Embulk Written in Java/JRuby http://www.slideshare.net/frsyuki/embuk-making-data-integration-works-relaxed http://www.embulk.org/
  • 7. Data Analytics Platform Data Analytics Service
  • 8.
  • 9.
  • 13. Data Analytics Flow Collect Store Process Visualize Data source Reporting Monitoring
  • 14. Data Analytics Flow Collect Store Process Visualize Data source Reporting Monitoring
  • 15. Data Analytics Platform • Data collection, storage • Console & API endpoints • Schema management • Processing (batch, query, ...) • Queuing & Scheduling • Data connector/exporter
  • 17. Data Analytics Platform • Data collection, storage: Ruby(OSS), Java/JRuby(OSS) • Console & API endpoints: Ruby(RoR) • Schema management: Ruby/Java (MessagePack) • Processing (batch, query, ...): Java(Hadoop,Presto) • Queuing & Scheduling: Ruby(OSS) • Data connector/exporter: Java, Java/JRuby(OSS)
  • 18. Treasure Data Architecture: Overview http://www.slideshare.net/tagomoris/data-analytics-service-company-and-its-ruby-usage Console API EventCollector PlazmaDB Worker Hadoop Cluster Presto Cluster USERS TD SDKs SERVERS DataConnector CUSTOMER's SYSTEMS Scheduler
  • 22. Queue/Worker, Scheduler • Treasure Data: multi-tenant data analytics service • executes many jobs in shared clusters (queries, imports, ...) • CORE: queues-workers & schedulers • Clusters have queues/scheduler... it's not enough • resource limitations for each price plans • priority queues for job types • and many others
  • 24. PerfectSched • Provides periodical/scheduled queries for customers • it's like reliable "cron" • Highly available distributed scheduler using RDBMS • Written in CRuby • At-least-once semantics • PerfectSched enqueues jobs into PerfectQueue
  • 25. Jobs in TD LOST Duplicated Retried for Errors Throughput Execution time DATA import/ export NG NG OK or NG HIGH SHORT (secs-mins) QUERY NG OK OK LOW SHORT (secs) or LONG (mins-hours)
  • 27. Worker Worker Worker Worker PerfectQueue overview Worker MySQL 1 table for 1 queue workers for queues hypervisor process worker processes worker processes worker processes
  • 28. Features • Priorities for query types • Resource limits per accounts • Graceful restarts • Queries must run long time (<= 1d) • New worker code should be loaded, besides running job with older code
  • 29. PerfectQueue • Highly available distributed queue using RDBMS • Enqueue by INSERT INTO • Dequeue/Commit by UPDATE • Using transactions • Flexible scheduling rather than scalability • Workers does many things • Plazmadb operations (including importing data) • Building job parameters • Handling results of jobs + kicking other jobs • Using Amazon RDS (MySQL) internally (+ Workers on EC2)
  • 30. Builing Jobs/Parameters • Parameters • for job types, accounts, price plans and clusters • to control performance/parallelism, permissions and data types • ex: Java properties • Jobs • to prepare for customers' queries • to make queries safer/faster • ex: Hive Queries (HiveQL, a variety of SQL)
  • 31. Example: Hive job env HADOOP_CLASSPATH=test.jar:td-hadoop-1.0.jar HADOOP_OPTS="-Xmx738m -Duser.name=221" hive --service jar td-hadoop-1.0.jar com.treasure_data.hadoop.hive.runner.QueryRunner -hiveconf td.jar.version= -hiveconf plazma.metadb.config={} -hiveconf plazma.storage.config={} -hiveconf td.worker.database.config={} -hiveconf mapreduce.job.priority=HIGH -hiveconf mapreduce.job.queuename=root.q221.high -hiveconf mapreduce.job.name=HiveJob379515 -hiveconf td.query.mergeThreshold=1333382400 -hiveconf td.query.apikey=12345 -hiveconf td.scheduled.time=1342449253 -hiveconf td.outdir=./jobs/379515 -hiveconf hive.metastore.warehouse.dir=/user/hive/221/warehouse -hiveconf hive.auto.convert.join.noconditionaltask=false -hiveconf hive.mapjoin.localtask.max.memory.usage=0.7 -hiveconf hive.mapjoin.smalltable.filesize=25000000 -hiveconf hive.resultset.use.unique.column.names=false -hiveconf hive.auto.convert.join=false -hiveconf hive.optimize.sort.dynamic.partition=false -hiveconf mapreduce.job.reduces=-1 -hiveconf hive.vectorized.execution.enabled=false -hiveconf mapreduce.job.ubertask.enable=true -hiveconf yarn.app.mapreduce.am.resource.mb=2048
  • 32. env HADOOP_CLASSPATH=test.jar:td-hadoop-1.0.jar HADOOP_OPTS="-Xmx738m -Duser.name=221" hive --service jar td-hadoop-1.0.jar com.treasure_data.hadoop.hive.runner.QueryRunner -hiveconf td.jar.version= -hiveconf plazma.metadb.config={} -hiveconf plazma.storage.config={} -hiveconf td.worker.database.config={} -hiveconf mapreduce.job.priority=HIGH -hiveconf mapreduce.job.queuename=root.q221.high -hiveconf mapreduce.job.name=HiveJob379515 -hiveconf td.query.mergeThreshold=1333382400 -hiveconf td.query.apikey=12345 -hiveconf td.scheduled.time=1342449253 -hiveconf td.outdir=./jobs/379515 -hiveconf hive.metastore.warehouse.dir=/user/hive/221/warehouse -hiveconf hive.auto.convert.join.noconditionaltask=false -hiveconf hive.mapjoin.localtask.max.memory.usage=0.7 -hiveconf hive.mapjoin.smalltable.filesize=25000000 -hiveconf hive.resultset.use.unique.column.names=false -hiveconf hive.auto.convert.join=false -hiveconf hive.optimize.sort.dynamic.partition=false -hiveconf mapreduce.job.reduces=-1 -hiveconf hive.vectorized.execution.enabled=false -hiveconf mapreduce.job.ubertask.enable=true -hiveconf yarn.app.mapreduce.am.resource.mb=2048 -hiveconf mapreduce.job.ubertask.maxmaps=1 -hiveconf mapreduce.job.ubertask.maxreduces=1 -hiveconf mapreduce.job.ubertask.maxbytes=536870912 -hiveconf td.hive.insertInto.dynamic.partitioning=false -outdir ./jobs/379515
  • 33. Example: Hive job (cont) ADD JAR 'td-hadoop-1.0.jar'; CREATE DATABASE IF NOT EXISTS `db`; USE `db`; CREATE TABLE tagomoris (`v` MAP<STRING,STRING>, `time` INT) STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler' WITH SERDEPROPERTIES ('msgpack.columns.mapping'='*,time') TBLPROPERTIES ( 'td.storage.user'='221', 'td.storage.database'='dfc', 'td.storage.table'='users_20100604_080812_ce9203d0', 'td.storage.path'='221/dfc/users_20100604_080812_ce9203d0', 'td.table_id'='2', 'td.modifiable'='true', 'plazma.data_set.name'='221/dfc/users_20100604_080812_ce9203d0' ); CREATE TABLE tbl1 ( `uid` INT, `key` STRING, `time` INT ) STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler' WITH SERDEPROPERTIES ('msgpack.columns.mapping'='uid,key,time') TBLPROPERTIES ( 'td.storage.user'='221', 'td.storage.database'='dfc',
  • 34. ADD JAR 'td-hadoop-1.0.jar'; CREATE DATABASE IF NOT EXISTS `db`; USE `db`; CREATE TABLE tagomoris (`v` MAP<STRING,STRING>, `time` INT) STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler' WITH SERDEPROPERTIES ('msgpack.columns.mapping'='*,time') TBLPROPERTIES ( 'td.storage.user'='221', 'td.storage.database'='dfc', 'td.storage.table'='users_20100604_080812_ce9203d0', 'td.storage.path'='221/dfc/users_20100604_080812_ce9203d0', 'td.table_id'='2', 'td.modifiable'='true', 'plazma.data_set.name'='221/dfc/users_20100604_080812_ce9203d0' ); CREATE TABLE tbl1 ( `uid` INT, `key` STRING, `time` INT ) STORED BY 'com.treasure_data.hadoop.hive.mapred.TDStorageHandler' WITH SERDEPROPERTIES ('msgpack.columns.mapping'='uid,key,time') TBLPROPERTIES ( 'td.storage.user'='221', 'td.storage.database'='dfc', 'td.storage.table'='contests_20100606_120720_96abe81a', 'td.storage.path'='221/dfc/contests_20100606_120720_96abe81a', 'td.table_id'='4', 'td.modifiable'='true', 'plazma.data_set.name'='221/dfc/contests_20100606_120720_96abe81a' ); USE `db`;
  • 35. USE `db`; CREATE TEMPORARY FUNCTION MSGPACK_SERIALIZE AS 'com.treasure_data.hadoop.hive.udf.MessagePackSerialize'; CREATE TEMPORARY FUNCTION TD_TIME_RANGE AS 'com.treasure_data.hadoop.hive.udf.GenericUDFTimeRange'; CREATE TEMPORARY FUNCTION TD_TIME_ADD AS 'com.treasure_data.hadoop.hive.udf.UDFTimeAdd'; CREATE TEMPORARY FUNCTION TD_TIME_FORMAT AS 'com.treasure_data.hadoop.hive.udf.UDFTimeFormat'; CREATE TEMPORARY FUNCTION TD_TIME_PARSE AS 'com.treasure_data.hadoop.hive.udf.UDFTimeParse'; CREATE TEMPORARY FUNCTION TD_SCHEDULED_TIME AS 'com.treasure_data.hadoop.hive.udf.GenericUDFScheduledTime'; CREATE TEMPORARY FUNCTION TD_X_RANK AS 'com.treasure_data.hadoop.hive.udf.Rank'; CREATE TEMPORARY FUNCTION TD_FIRST AS 'com.treasure_data.hadoop.hive.udf.GenericUDAFFirst'; CREATE TEMPORARY FUNCTION TD_LAST AS 'com.treasure_data.hadoop.hive.udf.GenericUDAFLast'; CREATE TEMPORARY FUNCTION TD_SESSIONIZE AS 'com.treasure_data.hadoop.hive.udf.UDFSessionize'; CREATE TEMPORARY FUNCTION TD_PARSE_USER_AGENT AS 'com.treasure_data.hadoop.hive.udf.GenericUDFParseUserAgent'; CREATE TEMPORARY FUNCTION TD_HEX2NUM AS 'com.treasure_data.hadoop.hive.udf.UDFHex2num'; CREATE TEMPORARY FUNCTION TD_MD5 AS 'com.treasure_data.hadoop.hive.udf.UDFmd5'; CREATE TEMPORARY FUNCTION TD_RANK_SEQUENCE AS 'com.treasure_data.hadoop.hive.udf.UDFRankSequence'; CREATE TEMPORARY FUNCTION TD_STRING_EXPLODER AS 'com.treasure_data.hadoop.hive.udf.GenericUDTFStringExploder'; CREATE TEMPORARY FUNCTION TD_URL_DECODE AS
  • 36. CREATE TEMPORARY FUNCTION TD_URL_DECODE AS 'com.treasure_data.hadoop.hive.udf.UDFUrlDecode'; CREATE TEMPORARY FUNCTION TD_DATE_TRUNC AS 'com.treasure_data.hadoop.hive.udf.UDFDateTrunc'; CREATE TEMPORARY FUNCTION TD_LAT_LONG_TO_COUNTRY AS 'com.treasure_data.hadoop.hive.udf.UDFLatLongToCountry'; CREATE TEMPORARY FUNCTION TD_SUBSTRING_INENCODING AS 'com.treasure_data.hadoop.hive.udf.GenericUDFSubstringInEncoding'; CREATE TEMPORARY FUNCTION TD_DIVIDE AS 'com.treasure_data.hadoop.hive.udf.GenericUDFDivide'; CREATE TEMPORARY FUNCTION TD_SUMIF AS 'com.treasure_data.hadoop.hive.udf.GenericUDAFSumIf'; CREATE TEMPORARY FUNCTION TD_AVGIF AS 'com.treasure_data.hadoop.hive.udf.GenericUDAFAvgIf'; CREATE TEMPORARY FUNCTION hivemall_version AS 'hivemall.HivemallVersionUDF'; CREATE TEMPORARY FUNCTION perceptron AS 'hivemall.classifier.PerceptronUDTF'; CREATE TEMPORARY FUNCTION train_perceptron AS 'hivemall.classifier.PerceptronUDTF'; CREATE TEMPORARY FUNCTION train_pa AS 'hivemall.classifier.PassiveAggressiveUDTF'; CREATE TEMPORARY FUNCTION train_pa1 AS 'hivemall.classifier.PassiveAggressiveUDTF'; CREATE TEMPORARY FUNCTION train_pa2 AS 'hivemall.classifier.PassiveAggressiveUDTF'; CREATE TEMPORARY FUNCTION train_cw AS 'hivemall.classifier.ConfidenceWeightedUDTF'; CREATE TEMPORARY FUNCTION train_arow AS 'hivemall.classifier.AROWClassifierUDTF'; CREATE TEMPORARY FUNCTION train_arowh AS 'hivemall.classifier.AROWClassifierUDTF';
  • 37. CREATE TEMPORARY FUNCTION train_arowh AS 'hivemall.classifier.AROWClassifierUDTF'; CREATE TEMPORARY FUNCTION train_scw AS 'hivemall.classifier.SoftConfideceWeightedUDTF'; CREATE TEMPORARY FUNCTION train_scw2 AS 'hivemall.classifier.SoftConfideceWeightedUDTF'; CREATE TEMPORARY FUNCTION adagrad_rda AS 'hivemall.classifier.AdaGradRDAUDTF'; CREATE TEMPORARY FUNCTION train_adagrad_rda AS 'hivemall.classifier.AdaGradRDAUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_perceptron AS 'hivemall.classifier.multiclass.MulticlassPerceptronUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_pa AS 'hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_pa1 AS 'hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_pa2 AS 'hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_cw AS 'hivemall.classifier.multiclass.MulticlassConfidenceWeightedUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_arow AS 'hivemall.classifier.multiclass.MulticlassAROWClassifierUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_scw AS 'hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF'; CREATE TEMPORARY FUNCTION train_multiclass_scw2 AS 'hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF'; CREATE TEMPORARY FUNCTION cosine_similarity AS 'hivemall.knn.similarity.CosineSimilarityUDF'; CREATE TEMPORARY FUNCTION cosine_sim AS 'hivemall.knn.similarity.CosineSimilarityUDF'; CREATE TEMPORARY FUNCTION jaccard AS 'hivemall.knn.similarity.JaccardIndexUDF';
  • 38. CREATE TEMPORARY FUNCTION jaccard AS 'hivemall.knn.similarity.JaccardIndexUDF'; CREATE TEMPORARY FUNCTION jaccard_similarity AS 'hivemall.knn.similarity.JaccardIndexUDF'; CREATE TEMPORARY FUNCTION angular_similarity AS 'hivemall.knn.similarity.AngularSimilarityUDF'; CREATE TEMPORARY FUNCTION euclid_similarity AS 'hivemall.knn.similarity.EuclidSimilarity'; CREATE TEMPORARY FUNCTION distance2similarity AS 'hivemall.knn.similarity.Distance2SimilarityUDF'; CREATE TEMPORARY FUNCTION hamming_distance AS 'hivemall.knn.distance.HammingDistanceUDF'; CREATE TEMPORARY FUNCTION popcnt AS 'hivemall.knn.distance.PopcountUDF'; CREATE TEMPORARY FUNCTION kld AS 'hivemall.knn.distance.KLDivergenceUDF'; CREATE TEMPORARY FUNCTION euclid_distance AS 'hivemall.knn.distance.EuclidDistanceUDF'; CREATE TEMPORARY FUNCTION cosine_distance AS 'hivemall.knn.distance.CosineDistanceUDF'; CREATE TEMPORARY FUNCTION angular_distance AS 'hivemall.knn.distance.AngularDistanceUDF'; CREATE TEMPORARY FUNCTION jaccard_distance AS 'hivemall.knn.distance.JaccardDistanceUDF'; CREATE TEMPORARY FUNCTION manhattan_distance AS 'hivemall.knn.distance.ManhattanDistanceUDF'; CREATE TEMPORARY FUNCTION minkowski_distance AS 'hivemall.knn.distance.MinkowskiDistanceUDF'; CREATE TEMPORARY FUNCTION minhashes AS 'hivemall.knn.lsh.MinHashesUDF'; CREATE TEMPORARY FUNCTION minhash AS 'hivemall.knn.lsh.MinHashUDTF'; CREATE TEMPORARY FUNCTION bbit_minhash AS 'hivemall.knn.lsh.bBitMinHashUDF'; CREATE TEMPORARY FUNCTION voted_avg AS 'hivemall.ensemble.bagging.VotedAvgUDAF';
  • 39. CREATE TEMPORARY FUNCTION voted_avg AS 'hivemall.ensemble.bagging.VotedAvgUDAF'; CREATE TEMPORARY FUNCTION weight_voted_avg AS 'hivemall.ensemble.bagging.WeightVotedAvgUDAF'; CREATE TEMPORARY FUNCTION wvoted_avg AS 'hivemall.ensemble.bagging.WeightVotedAvgUDAF'; CREATE TEMPORARY FUNCTION max_label AS 'hivemall.ensemble.MaxValueLabelUDAF'; CREATE TEMPORARY FUNCTION maxrow AS 'hivemall.ensemble.MaxRowUDAF'; CREATE TEMPORARY FUNCTION argmin_kld AS 'hivemall.ensemble.ArgminKLDistanceUDAF'; CREATE TEMPORARY FUNCTION mhash AS 'hivemall.ftvec.hashing.MurmurHash3UDF'; CREATE TEMPORARY FUNCTION sha1 AS 'hivemall.ftvec.hashing.Sha1UDF'; CREATE TEMPORARY FUNCTION array_hash_values AS 'hivemall.ftvec.hashing.ArrayHashValuesUDF'; CREATE TEMPORARY FUNCTION prefixed_hash_values AS 'hivemall.ftvec.hashing.ArrayPrefixedHashValuesUDF'; CREATE TEMPORARY FUNCTION polynomial_features AS 'hivemall.ftvec.pairing.PolynomialFeaturesUDF'; CREATE TEMPORARY FUNCTION powered_features AS 'hivemall.ftvec.pairing.PoweredFeaturesUDF'; CREATE TEMPORARY FUNCTION rescale AS 'hivemall.ftvec.scaling.RescaleUDF'; CREATE TEMPORARY FUNCTION rescale_fv AS 'hivemall.ftvec.scaling.RescaleUDF'; CREATE TEMPORARY FUNCTION zscore AS 'hivemall.ftvec.scaling.ZScoreUDF'; CREATE TEMPORARY FUNCTION normalize AS 'hivemall.ftvec.scaling.L2NormalizationUDF'; CREATE TEMPORARY FUNCTION conv2dense AS 'hivemall.ftvec.conv.ConvertToDenseModelUDAF'; CREATE TEMPORARY FUNCTION to_dense_features AS 'hivemall.ftvec.conv.ToDenseFeaturesUDF';
  • 40. CREATE TEMPORARY FUNCTION to_dense_features AS 'hivemall.ftvec.conv.ToDenseFeaturesUDF'; CREATE TEMPORARY FUNCTION to_dense AS 'hivemall.ftvec.conv.ToDenseFeaturesUDF'; CREATE TEMPORARY FUNCTION to_sparse_features AS 'hivemall.ftvec.conv.ToSparseFeaturesUDF'; CREATE TEMPORARY FUNCTION to_sparse AS 'hivemall.ftvec.conv.ToSparseFeaturesUDF'; CREATE TEMPORARY FUNCTION quantify AS 'hivemall.ftvec.conv.QuantifyColumnsUDTF'; CREATE TEMPORARY FUNCTION vectorize_features AS 'hivemall.ftvec.trans.VectorizeFeaturesUDF'; CREATE TEMPORARY FUNCTION categorical_features AS 'hivemall.ftvec.trans.CategoricalFeaturesUDF'; CREATE TEMPORARY FUNCTION indexed_features AS 'hivemall.ftvec.trans.IndexedFeatures'; CREATE TEMPORARY FUNCTION quantified_features AS 'hivemall.ftvec.trans.QuantifiedFeaturesUDTF'; CREATE TEMPORARY FUNCTION quantitative_features AS 'hivemall.ftvec.trans.QuantitativeFeaturesUDF'; CREATE TEMPORARY FUNCTION amplify AS 'hivemall.ftvec.amplify.AmplifierUDTF'; CREATE TEMPORARY FUNCTION rand_amplify AS 'hivemall.ftvec.amplify.RandomAmplifierUDTF'; CREATE TEMPORARY FUNCTION addBias AS 'hivemall.ftvec.AddBiasUDF'; CREATE TEMPORARY FUNCTION add_bias AS 'hivemall.ftvec.AddBiasUDF'; CREATE TEMPORARY FUNCTION sortByFeature AS 'hivemall.ftvec.SortByFeatureUDF'; CREATE TEMPORARY FUNCTION sort_by_feature AS 'hivemall.ftvec.SortByFeatureUDF'; CREATE TEMPORARY FUNCTION extract_feature AS 'hivemall.ftvec.ExtractFeatureUDF';
  • 41. CREATE TEMPORARY FUNCTION extract_feature AS 'hivemall.ftvec.ExtractFeatureUDF'; CREATE TEMPORARY FUNCTION extract_weight AS 'hivemall.ftvec.ExtractWeightUDF'; CREATE TEMPORARY FUNCTION add_feature_index AS 'hivemall.ftvec.AddFeatureIndexUDF'; CREATE TEMPORARY FUNCTION feature AS 'hivemall.ftvec.FeatureUDF'; CREATE TEMPORARY FUNCTION feature_index AS 'hivemall.ftvec.FeatureIndexUDF'; CREATE TEMPORARY FUNCTION tf AS 'hivemall.ftvec.text.TermFrequencyUDAF'; CREATE TEMPORARY FUNCTION train_logregr AS 'hivemall.regression.LogressUDTF'; CREATE TEMPORARY FUNCTION train_pa1_regr AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION train_pa1a_regr AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION train_pa2_regr AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION train_pa2a_regr AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION train_arow_regr AS 'hivemall.regression.AROWRegressionUDTF'; CREATE TEMPORARY FUNCTION train_arowe_regr AS 'hivemall.regression.AROWRegressionUDTF'; CREATE TEMPORARY FUNCTION train_arowe2_regr AS 'hivemall.regression.AROWRegressionUDTF'; CREATE TEMPORARY FUNCTION train_adagrad_regr AS 'hivemall.regression.AdaGradUDTF'; CREATE TEMPORARY FUNCTION train_adadelta_regr AS 'hivemall.regression.AdaDeltaUDTF'; CREATE TEMPORARY FUNCTION train_adagrad AS 'hivemall.regression.AdaGradUDTF';
  • 42. CREATE TEMPORARY FUNCTION train_adagrad AS 'hivemall.regression.AdaGradUDTF'; CREATE TEMPORARY FUNCTION train_adadelta AS 'hivemall.regression.AdaDeltaUDTF'; CREATE TEMPORARY FUNCTION logress AS 'hivemall.regression.LogressUDTF'; CREATE TEMPORARY FUNCTION pa1_regress AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION pa1a_regress AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION pa2_regress AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION pa2a_regress AS 'hivemall.regression.PassiveAggressiveRegressionUDTF'; CREATE TEMPORARY FUNCTION arow_regress AS 'hivemall.regression.AROWRegressionUDTF'; CREATE TEMPORARY FUNCTION arowe_regress AS 'hivemall.regression.AROWRegressionUDTF'; CREATE TEMPORARY FUNCTION arowe2_regress AS 'hivemall.regression.AROWRegressionUDTF'; CREATE TEMPORARY FUNCTION adagrad AS 'hivemall.regression.AdaGradUDTF'; CREATE TEMPORARY FUNCTION adadelta AS 'hivemall.regression.AdaDeltaUDTF'; CREATE TEMPORARY FUNCTION float_array AS 'hivemall.tools.array.AllocFloatArrayUDF'; CREATE TEMPORARY FUNCTION array_remove AS 'hivemall.tools.array.ArrayRemoveUDF'; CREATE TEMPORARY FUNCTION sort_and_uniq_array AS 'hivemall.tools.array.SortAndUniqArrayUDF'; CREATE TEMPORARY FUNCTION subarray_endwith AS 'hivemall.tools.array.SubarrayEndWithUDF'; CREATE TEMPORARY FUNCTION subarray_startwith AS 'hivemall.tools.array.SubarrayStartWithUDF'; CREATE TEMPORARY FUNCTION collect_all AS
  • 43. CREATE TEMPORARY FUNCTION collect_all AS 'hivemall.tools.array.CollectAllUDAF'; CREATE TEMPORARY FUNCTION concat_array AS 'hivemall.tools.array.ConcatArrayUDF'; CREATE TEMPORARY FUNCTION subarray AS 'hivemall.tools.array.SubarrayUDF'; CREATE TEMPORARY FUNCTION array_avg AS 'hivemall.tools.array.ArrayAvgGenericUDAF'; CREATE TEMPORARY FUNCTION array_sum AS 'hivemall.tools.array.ArraySumUDAF'; CREATE TEMPORARY FUNCTION to_string_array AS 'hivemall.tools.array.ToStringArrayUDF'; CREATE TEMPORARY FUNCTION map_get_sum AS 'hivemall.tools.map.MapGetSumUDF'; CREATE TEMPORARY FUNCTION map_tail_n AS 'hivemall.tools.map.MapTailNUDF'; CREATE TEMPORARY FUNCTION to_map AS 'hivemall.tools.map.UDAFToMap'; CREATE TEMPORARY FUNCTION to_ordered_map AS 'hivemall.tools.map.UDAFToOrderedMap'; CREATE TEMPORARY FUNCTION sigmoid AS 'hivemall.tools.math.SigmoidGenericUDF'; CREATE TEMPORARY FUNCTION taskid AS 'hivemall.tools.mapred.TaskIdUDF'; CREATE TEMPORARY FUNCTION jobid AS 'hivemall.tools.mapred.JobIdUDF'; CREATE TEMPORARY FUNCTION rowid AS 'hivemall.tools.mapred.RowIdUDF'; CREATE TEMPORARY FUNCTION generate_series AS 'hivemall.tools.GenerateSeriesUDTF'; CREATE TEMPORARY FUNCTION convert_label AS 'hivemall.tools.ConvertLabelUDF'; CREATE TEMPORARY FUNCTION x_rank AS 'hivemall.tools.RankSequenceUDF'; CREATE TEMPORARY FUNCTION each_top_k AS 'hivemall.tools.EachTopKUDTF'; CREATE TEMPORARY FUNCTION tokenize AS 'hivemall.tools.text.TokenizeUDF'; CREATE TEMPORARY FUNCTION is_stopword AS 'hivemall.tools.text.StopwordUDF'; CREATE TEMPORARY FUNCTION split_words AS
  • 44. CREATE TEMPORARY FUNCTION split_words AS 'hivemall.tools.text.SplitWordsUDF'; CREATE TEMPORARY FUNCTION normalize_unicode AS 'hivemall.tools.text.NormalizeUnicodeUDF'; CREATE TEMPORARY FUNCTION lr_datagen AS 'hivemall.dataset.LogisticRegressionDataGeneratorUDTF'; CREATE TEMPORARY FUNCTION f1score AS 'hivemall.evaluation.FMeasureUDAF'; CREATE TEMPORARY FUNCTION mae AS 'hivemall.evaluation.MeanAbsoluteErrorUDAF'; CREATE TEMPORARY FUNCTION mse AS 'hivemall.evaluation.MeanSquaredErrorUDAF'; CREATE TEMPORARY FUNCTION rmse AS 'hivemall.evaluation.RootMeanSquaredErrorUDAF'; CREATE TEMPORARY FUNCTION mf_predict AS 'hivemall.mf.MFPredictionUDF'; CREATE TEMPORARY FUNCTION train_mf_sgd AS 'hivemall.mf.MatrixFactorizationSGDUDTF'; CREATE TEMPORARY FUNCTION train_mf_adagrad AS 'hivemall.mf.MatrixFactorizationAdaGradUDTF'; CREATE TEMPORARY FUNCTION fm_predict AS 'hivemall.fm.FMPredictGenericUDAF'; CREATE TEMPORARY FUNCTION train_fm AS 'hivemall.fm.FactorizationMachineUDTF'; CREATE TEMPORARY FUNCTION train_randomforest_classifier AS 'hivemall.smile.classification.RandomForestClassifierUDTF'; CREATE TEMPORARY FUNCTION train_rf_classifier AS 'hivemall.smile.classification.RandomForestClassifierUDTF'; CREATE TEMPORARY FUNCTION train_randomforest_regr AS 'hivemall.smile.regression.RandomForestRegressionUDTF'; CREATE TEMPORARY FUNCTION train_rf_regr AS 'hivemall.smile.regression.RandomForestRegressionUDTF'; CREATE TEMPORARY FUNCTION tree_predict AS 'hivemall.smile.tools.TreePredictByStackMachineUDF';
  • 45. CREATE TEMPORARY FUNCTION tree_predict AS 'hivemall.smile.tools.TreePredictByStackMachineUDF'; CREATE TEMPORARY FUNCTION vm_tree_predict AS 'hivemall.smile.tools.TreePredictByStackMachineUDF'; CREATE TEMPORARY FUNCTION rf_ensemble AS 'hivemall.smile.tools.RandomForestEnsembleUDAF'; CREATE TEMPORARY FUNCTION train_gradient_boosting_classifier AS 'hivemall.smile.classification.GradientTreeBoostingClassifierUDTF'; CREATE TEMPORARY FUNCTION guess_attribute_types AS 'hivemall.smile.tools.GuessAttributesUDF'; CREATE TEMPORARY FUNCTION tokenize_ja AS 'hivemall.nlp.tokenizer.KuromojiUDF'; CREATE TEMPORARY MACRO max2(x DOUBLE, y DOUBLE) if(x>y,x,y); CREATE TEMPORARY MACRO min2(x DOUBLE, y DOUBLE) if(x<y,x,y); CREATE TEMPORARY MACRO rand_gid(k INT) floor(rand()*k); CREATE TEMPORARY MACRO rand_gid2(k INT, seed INT) floor(rand(seed)*k); CREATE TEMPORARY MACRO idf(df_t DOUBLE, n_docs DOUBLE) log(10, n_docs / max2(1,df_t)) + 1.0; CREATE TEMPORARY MACRO tfidf(tf FLOAT, df_t DOUBLE, n_docs DOUBLE) tf * (log(10, n_docs / max2(1,df_t)) + 1.0); SELECT time, COUNT(1) AS cnt FROM tbl1 WHERE TD_TIME_RANGE(time, '2015-12-11', '2015-12-12', 'JST');
  • 46. Do you still love Java / SQL ?
  • 47. PQ written in Ruby • Building jobs/parameters is so complex! • using data from many configurations (YAML, JSON), internal APIs and RDBMSs • with many ext syntaxes/rules to tune performance, override configurations for tests, ... • Ruby empower to write fat/complex worker code • Testing! • Unit tests using Rspec • System tests (executing real queries/jobs) using Rspec
  • 48. For Further improvement about workers • More performance for more customers and less costs • More scalability for many other kind jobs • Better and well-controlled tests (indented here documents!)
  • 49. "Done is better than Perfect."
  • 51. We'll improve our code step by step, with improvements of ruby and developer community <3 Thanks!