Coder Social home page Coder Social logo

gojek / ziggurat Goto Github PK

View Code? Open in Web Editor NEW
394.0 33.0 64.0 1.46 MB

A stream processing framework to build stateless applications on Kafka

License: Apache License 2.0

Clojure 92.75% Java 6.96% Makefile 0.27% Shell 0.01%
kafka-streams clojure framework

ziggurat's Introduction

Ziggurat

Build Status Coverage Status Clojars Project

Table of Contents

Description

Ziggurat is a framework built to simplify stream processing on Kafka. It can be used to create a full-fledged Clojure app that reads and processes messages from Kafka. Ziggurat abstracts the following features:

  • Reading messages from Kafka
  • Retrying failed messages via RabbitMQ
  • Setting up an HTTP server

Refer to concepts to understand the concepts referred to in this document.

Important Concepts and Usage Docs

Dev Setup

For Mac Users Only

  1. Install Clojure: brew install clojure
  2. Install Leiningen: brew install leiningen
  3. Run Docker Compose: docker-compose up. This starts:
  • Kafka on localhost:9092
  • ZooKeeper on localhost:2181
  • RabbitMQ on localhost:5672
  1. Run tests: make test

Running a Cluster Setup Locally

  • Run make setup-cluster. This clears up the volume and starts:
    • 3 Kafka brokers on localhost:9091, localhost:9092, and localhost:9093
    • Zookeeper on localhost:2181
    • RabbitMQ on localhost:5672

Running Tests via a Cluster

  • Run make test-cluster. This uses config.test.cluster.edn instead of config.test.edn.

Usage

Add this to your project.clj:

[tech.gojek/ziggurat "4.11.1"]


_Please refer [clojars](https://clojars.org/tech.gojek/ziggurat) for the latest stable version_

To start a stream (a thread that reads messages from Kafka), add this to your core namespace.

```clojure
(require '[ziggurat.init :as ziggurat])

(defn start-fn []
    ;; your logic that runs at startup goes here
)

(defn stop-fn []
    ;; your logic that runs at shutdown goes here
)

(defn main-fn
  [{:keys [message metadata] :as message-payload}]
    (println message)
    :success)

(def handler-fn
    (-> main-fn
      (middleware/protobuf->hash ProtoClass :stream-id)))
;; Here ProtoClass refers to the fully qualified name of the Java class which the code is used to de-serialize the message.

(ziggurat/main start-fn stop-fn {:stream-id {:handler-fn handler-fn}})

NOTE: this example assumes that the message is serialized in Protobuf format

Please refer the Middleware section for understanding handler-fn here.

  • The main-fn is the function that will be applied to every message that is read from the Kafka stream.
  • The main-fn will take map as an argument that takes 2 keys i.e
    • message - It is the byte[] array received from kafka.
    • metadata
      • topic - It is the topic from where kafka message is consumed.
      • timestamp - It is ingestion timestamp in kafka.
      • partition - The partition from message is consumed.
      • rabbitmq-retry-count - The number of retries done by rabbitmq for given message.
  • The main-fn returns a keyword which can be any of the below words
    • :success - The message was successfully processed and the stream should continue to the next message
    • :retry - The message failed to be processed and it should be retried via RabbitMQ.
    • :dead-letter - The message is not retried and is directly pushed to the dead letter queue
    • :skip - The message should be skipped without reporting its failure or retrying the message. Same as :success except that a different metric is published to track skipped messages
  • The start-fn is run at the application startup and can be used to initialize connection to databases, http clients, thread-pools, etc.
  • The stop-fn is run at shutdown and facilitates graceful shutdown, for example, releasing db connections, shutting down http servers etc.

Multiple stream routes

  • Ziggurat enables reading from multiple streams and applying same/different functions to the messages. :stream-id is a unique identifier per stream which needs to be included in config.edn file
  • All configs, queues and metrics will be namespaced under this id.
(ziggurat/main start-fn stop-fn {:stream-id-1 {:handler-fn handler-fn-1}
                                         :stream-id-2 {:handler-fn handler-fn-2}})
(require '[ziggurat.init :as ziggurat])

(defn start-fn []
    ;; your logic that runs at startup goes here
)

(defn stop-fn []
    ;; your logic that runs at shutdown goes here
)


(defn api-handler [_request]
  {:status  200
   :headers {"Content-Type" "application/json"}
   :body    (get-resource)})

(def routes [["v1/resources" {:get api-handler}]])

(defn main-fn
  [{:keys [message metadata] :as message-payload}]
    (println message)
    :success)

(def handler-fn
    (-> main-fn
      (middleware/protobuf->hash ProtoClass :stream-id)))

(ziggurat/main start-fn stop-fn {:stream-id {:handler-fn handler-fn}} routes)

NOTE: this example assumes that the message is serialized in Protobuf format

Deprecation Notice

  • Sentry has been deprecated from version 4.6.3.

Contribution

  • For dev setup and contributions please refer to CONTRIBUTING.md

License

Copyright 2018, GO-JEK Tech <http://gojek.tech>

Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at

    http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.

ziggurat's People

Contributors

againom avatar anmol1vw13 avatar gurupratap91 avatar hariuserx avatar harshalbhatia avatar hogaur avatar indrajithi avatar isomnath avatar kartik7153 avatar kitallis avatar krriteshgupta avatar led avatar macalimlim avatar mistrys47 avatar mjayprateek avatar msbond avatar nid90 avatar prashant3863 avatar rajnishdashora avatar roobalimsab avatar s7saxena avatar scimetfoo avatar shubhang93 avatar sisinduku avatar sohamkamani avatar sravankorumilli avatar theanirudhvyas avatar vruttant1403 avatar wickedbrat avatar williamchang80 avatar

Stargazers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

Watchers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

ziggurat's Issues

Wrong delay calculation

Currently Ziggurat is using WallclockTimestampExtractor stream processor. If you see WallclockTimestampExtractor definition

public class WallclockTimestampExtractor implements TimestampExtractor {

    /**
     * Return the current wall clock time as timestamp.
     *
     * @param record a data record
     * @param previousTimestamp the latest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
     * @return the current wall clock time, expressed in milliseconds since midnight, January 1, 1970 UTC
     */
    @Override
    public long extract(final ConsumerRecord<Object, Object> record, final long previousTimestamp) {
        return System.currentTimeMillis();
    }

In that case, the timestamp will be overridden by current timestamp. Hence when you call delay calculation in kafka_delay.clj:
(calculate-and-report-kafka-delay metric-namespace (.timestamp processor-context))
It will use current timestamp when it is invoked. It will make delay calculation incorrect

datadog is confusing is config.edn

Team,
config.edn comes up with a statds configuration with host and port that metrics should be sent to. But the key name is shown as datadog which is confusing.

Thanks,
Vidya.

Producing Support

Kafka Producer - Kafka streams give a Kafka producer but we cannot use it, due to retry functionality. We need to add a custom Kafka producer that exposes configurations.

README Improvements

This ticket captures all the improvements one wants to see in Ziggurat README. Please comment below with your wishlist item for Ziggurat README. All of those will be taken up as a part of this ticket.

  • Structuring of README should be improved. At present, it makes a lot of assumptions about the potential reader - knowledge of clojure, concept of streaming, Kafka, etc. But, that's not always true. Every README should be written targeting a novice (to Ziggurat and associated technologies, obviously).
    It should start with a brief introduction to Ziggurat, a section on important concepts with a short list on further reads, and an example driven step-by-step guide for getting an app up and running with all major features provided by Ziggurat - streams, routes, metrics, producer, testing, etc. And obviously, an upgrade guide. This structure is just my thought of a good readme. One should look at good external examples for inspiration.

  • A section on concepts would really help a newbie developing a good understanding of the components such as consuming from Kafka Streams and retries using RabbitMQ. At present, README clearly lacks any information about how retries and queues work in Ziggurat.

  • [Configuration] README should specify the recommended value for all the configurations in Ziggurat, which aligns with the best practices for the system/tool/framework. Also, every such recommendation should be accompanied by an external link which endorses such a recommendation.
    For example, Ziggurat based apps should configure stream-threads-count ideally as Number of total partitions/Number of stream tasks. Things like would help a newbie to avoid the usual pitfalls due to wrong configurations.

Support for experimental/alpha features in Ziggurat

Trunk based development in Ziggurat implies that not all Work-In-Progress features might be production-ready at the same time. Releases might be made with some experimental/alpha features which are not suitable to be used in a production environment.

Thus, there is a requirement in Ziggurat to:

  1. Mark certain features (parts of the code) as alpha (within the code or through configuration params)
  2. Facility to enable/disable those features by a simple flag or through runtime args
  3. All the experimental features can be disabled together or individually

This will enable all the experimental (alpha) features to co-exist with the stable ones in the master branch.

[BUG] Shutdown of service leads to loss of message which did not finish processing before shutdown

When there is a message for which the mapper function has started processing, and the service using ziggurat is shut down, the mapper function execution for the message stops without running the complete function.

When the service is started again, the message is not reprocessed by the service, leading to loss of the message. This issue was seen on ziggurat v2.12.4

Steps to reproduce

  1. Consume a message with a long execution time mapper-fn.
    Add a log at the start of the mapper fn and at the end of the mapper fn.
  2. Restart the service immediately.
  3. Message which did not fully process due to shutdown, will not be reprocessed by ziggurat.

Support Stream route disabling

This will be handy in case of disaster management. Few examples where this would be a helpful feature.
Suppose there are multiple stream routes in an actor .
1.) One of the routes in this actor starts messing up the data and we want to stop that immediately.

2.) One of the routes in the actor which is not business critical ends up doing some heavy data base operation because of which the processing of business critical events gets slow.We should be able to stop consumption of such route for the time being.

Currently there is no way to achieve this without deleting the route from the code.

So, as a user I should be able to control the enablement of a route by toggling on and off a config.

Custom Middleware support for http Server

Currently the middleware added to HTTP server is defined in ziggurat.server.routes/handler. If I want to add any custom middleware to the server, I have to update this namespace and hence raise a PR in ziggurat and then release it.

Report error to new relic in mapper.clj

Catch all is right now pushing error to sentry. Would like to publish the same to new relic as well. Planning to use noticeError provided by newrelic java agent.

analyse how to handle network failures for RabbitMQ.

Applications stop consuming messages in case RabbitMQ restarts or there are connectivity issues.

[ERROR] 2019-04-20 16:05:11,345 [AMQP Connection 127.0.0.1:5672] com.rabbitmq.client.impl.ForgivingExceptionHandler:log: Caught an exception when recovering topology Caught an exception while recovering binding between booking_dss_transport_instant_exchange and booking_dss_transport_instant_queue: channel is already closed due to clean channel shutdown; protocol method: #method<channel.close>(reply-code=200, reply-text=OK, class-id=0, method-id=0)
com.rabbitmq.client.TopologyRecoveryException: Caught an exception while recovering binding between booking_dss_transport_instant_exchange and booking_dss_transport_instant_queue: channel is already closed due to clean channel shutdown; protocol method: #method<channel.close>(reply-code=200, reply-text=OK, class-id=0, method-id=0)
    at com.rabbitmq.client.impl.recovery.AutorecoveringConnection.recoverBindings(AutorecoveringConnection.java:698) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.recovery.AutorecoveringConnection.recoverEntities(AutorecoveringConnection.java:637) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.recovery.AutorecoveringConnection.beginAutomaticRecovery(AutorecoveringConnection.java:559) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.recovery.AutorecoveringConnection.access$000(AutorecoveringConnection.java:58) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.recovery.AutorecoveringConnection$3.recoveryCanBegin(AutorecoveringConnection.java:488) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.AMQConnection.notifyRecoveryCanBeginListeners(AMQConnection.java:748) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.AMQConnection.doFinalShutdown(AMQConnection.java:725) [amqp-client-5.2.0.jar:5.2.0]
    at com.rabbitmq.client.impl.AMQConnection$MainLoop.run(AMQConnection.java:609) [amqp-client-5.2.0.jar:5.2.0]
    at java.lang.Thread.run(Thread.java:748) [?:1.8.0_131]

Does not retry messages from deadset when replayed

When a user replays a message from the deadset queue, Ziggurat processes the message only once. It does not send the message through a retry flow even if the mapper function returns the keyword :retry.

Ability to filter out messages by time

In Kafka streams, there is a concept of application IDs by which we seek the offset read the messages accordingly. To make our service idempotent, we can choose to ignore the messages based on time. From based on the time I mean relative time like ignore the messages that are before 15 mins.

Add metrics in line with recommendations from prometheus

Prometheus has this documentation page which lays out a general recommendation that we should we emitting duration related metrics in seconds, with a suffix _seconds (amongst other things)

Considering the role and ubiquity of prometheus, I feel it will be prudent to add metrics which comply with the best practices laid out. This will allow tools build atop prom, as well people who come expecting prom best practices to be followed, to automatically feel familiar with the metrics exposed by ziggurat.

Thoughts?

Running actor in different modes

Allow to only run in worker or server mode, as well as current default mode. Also moving actor management APIs from http routes. Advantages
Isolation between different goals of Kafka consumption and HTTP serving
Easier orchestration, as stream upgrade process can be significantly different from HTTP

[Analysis] Benchmark HTTP threads on Ziggurat

This comes from a user's feedback about Ziggurat:

We should audit number of HTTP threads of actors. If one of the downstream service is slow then all the threads are exhausted and consul check fails which de registers it from consul.

The task here is to analyze the following:

  1. At present, what's the maximum number of concurrent HTTP threads a Ziggurat app can support
  2. What are the reasons for exhaustion? Are there ways to make sure that app self-heals in such a case? For example, making async calls to downstream if the detected lag is more than a certain threshold (2-5 seconds, for example). This will help release threads faster.
  3. Explore network-level tweaks to ensure the underlying server can operate under high load.

Sharing some links I've found online.

https://www.wiliam.com.au/wiliam-blog/thread-exhaustion
https://wiki.eclipse.org/Jetty/Howto/High_Load

Kafka stream threads die when partitions of source topics get increased

We are using ziggurat for consuming messages from the topic and we need to increase the topic of the source topic. We saw that stream threads were dying with the exception message - couldn't create an internal topic. We know that Kafka streams cannot handle change in partitions somehow ziggurat can handle this error gracefully and recover from it without restarting the whole app. Any ideas suggestion welcome.

Retryable message persistence guarantees

In case I have my retries enabled and my rabbitMQ is unreachable(network partition, rabbitMQ failures, etc.), the messages that I am retrying will be lost. These messages might be critical as well. Currently, in ziggurat, we do retries in case of a failure to publish. Just wondering will it make sense to have local storage of some sorts which could persist messages and publish them to the designated queue once the normalcy is restored? https://www.rabbitmq.com/api-guide.html#publishers

Thoughts?

Middleware Support for message processing

Similar to most HTTP frameworks, middleware are common piece of code that stack up during message processing. this has several advantages

  • Decentralised middleware development
  • Leaner base framework
  • Organic way of incorporating “successful” middleware
  • higher flexibility
  • No unnecessary upgrades of framework, for features teams don’t want
    Almost all above ”message processing” problems can be tackled in this way.

ziggurat throws a null pointer exception if retries are not enabled and deadset messages are requessted

error trace:

"Error": "java.lang.NullPointerException: null
                     channel.clj:61 langohr.channel/open
                     channel.clj:55 langohr.channel/open
                    consumer.clj:48 ziggurat.messaging.consumer/get-dead-set-messages
                    consumer.clj:38 ziggurat.messaging.consumer/get-dead-set-messages
                    dead_set.clj:14 ziggurat.messaging.dead-set/view
                    dead_set.clj:11 ziggurat.messaging.dead-set/view
                    dead_set.clj:37 ziggurat.resource.dead-set/view
                    dead_set.clj:33 ziggurat.resource.dead-set/view
                       ring.cljc:25 bidi.ring$fn__11546.invokeStatic
                       ring.cljc:21 bidi.ring$fn__11546.invoke
                       ring.cljc:16 bidi.ring$fn__11531$G__11526__11540.invoke
                       ring.cljc:43 bidi.ring$make_handler$fn__11551.invoke
                  middleware.clj:22 ziggurat.server.middleware/wrap-hyphenate[fn]
              keyword_params.clj:36 ring.middleware.keyword-params/wrap-keyword-params[fn]
                      params.clj:67 ring.middleware.params/wrap-params[fn]
          absolute_redirects.clj:47 ring.middleware.absolute-redirects/wrap-absolute-redirects[fn]
                content_type.clj:34 ring.middleware.content-type/wrap-content-type[fn]
             default_charset.clj:31 ring.middleware.default-charset/wrap-default-charset[fn]
                not_modified.clj:53 ring.middleware.not-modified/wrap-not-modified[fn]
                        json.clj:74 ring.middleware.json/wrap-json-params[fn]
                        json.clj:87 ring.middleware.json/wrap-json-response[fn]
                  middleware.clj:13 ziggurat.server.middleware/wrap-default-content-type-json[fn]
                        ring.clj:92 new-reliquary.ring/web-transaction[fn]
                        core.clj:32 new-reliquary.core/new-reliquary.core.NewRelicTracer
                        core.clj:35 new-reliquary.core/new-reliquary.core.NewRelicTracer
                        core.clj:49 new-reliquary.core/with-newrelic-transaction
                        core.clj:43 new-reliquary.core/with-newrelic-transaction
                        ring.clj:99 new-reliquary.ring/wrap-newrelic-transaction[fn]
                  middleware.clj:30 ziggurat.server.middleware/wrap-errors[fn]
                      logger.clj:19 ring.logger/wrap-with-logger*[fn]
                      logger.clj:37 ring.logger/wrap-request-start[fn]
                       jetty.clj:25 ring.adapter.jetty/proxy-handler[fn]
                   (Unknown Source) ring.adapter.jetty.proxy$org.eclipse.jetty.server.handler.AbstractHandler$ff19274a.handle
             HandlerWrapper.java:97 org.eclipse.jetty.server.handler.HandlerWrapper.handle
                    Server.java:499 org.eclipse.jetty.server.Server.handle
               HttpChannel.java:311 org.eclipse.jetty.server.HttpChannel.handle
            HttpConnection.java:258 org.eclipse.jetty.server.HttpConnection.onFillable
        AbstractConnection.java:544 org.eclipse.jetty.io.AbstractConnection$2.run
          QueuedThreadPool.java:635 org.eclipse.jetty.util.thread.QueuedThreadPool.runJob
          QueuedThreadPool.java:555 org.eclipse.jetty.util.thread.QueuedThreadPool$3.run
                    Thread.java:748 java.lang.Thread.run

Sending metrics from start, stream-routes params causes failures

When sending metrics from start and stream-routes causes to failure of sending metrics as ziggurat/metrics initialization is happening later.

Here is the relevant code:

(defn start
  "Starts up Ziggurat's config, actor fn, rabbitmq connection and then streams, server etc"
  [actor-start-fn stream-routes actor-routes]
  (start* #{#'config/config})
  (actor-start-fn)
  (start* #{#'messaging-connection/connection} {:stream-routes stream-routes})
  (messaging-producer/make-queues stream-routes)
  (messaging-consumer/start-subscribers stream-routes)      ;; We want subscribers to start after creating queues on RabbitMQ.
  (start* #{#'statsd-reporter
            #'server/server
            #'nrepl-server/server
            #'streams/stream
            #'sentry-reporter}
          {:stream-routes stream-routes
           :actor-routes  actor-routes}))

Add custom timeout for messages in channel instant queue

Channels are meant for async processing of messages. When a message is pushed to the channel instant queue, there is no support to add a custom timeout to the message, so that we can add a delay to this processing.
It would be really helpful if we can add a custom delay while processing messages using channels.

Improving metrics by using simple-statsd

To quote some of the users of Ziggurat, they've noted the following problems with the way Ziggurat has published metrics:

Graphing a counter published from one of the libraries in [ziggurat.metrics, metrics-clojure or dropwizard-statsd] show a line indicating throughput like you'd expect. Instead it results in a monotonically increasing number which is reset to 0 whenever the process is restarted.

This problem has to do with the Dropwizard Metrics library that all of these other wrapper libraries depend on. Instead of using the native counter, gauge and timing abstractions that statsd provides out of the box, this library maintains in-app state to compute the value of a counter or the rate of a meter at any given moment, and publishes these numbers periodically to statsd as a gauge. I found this problematic because it meant that performing aggregation or rate computation in Grafana was more difficult or impossible.

Thus, there are two options here:

  1. Investigate if these problems can be rectified by proper use of dropwizard library
  2. Use a simple statsd library instead.

Allow passing Vars as a topic handler

The handler function passed to init/main requires it to a function. Ideally it shouldn't care about the concrete implementation, it should only care if the passed in argument satisfies the function interface IFn . Case in point, we can't pass a clojure var as a handler.

(defn- another-fn
  []
  (println "This code can be modified via REPL"))

(defn- my-handler-fn
  [& _]
  (println "This code can't be modified via REPL")
  (another-fn))

(init/main start mount/stop
  {:topic {:handler-fn my-handler-fn}
  ;; :topic {:handler-fn #'my-handler-fn} - this throws an error  
  })

API to clear dead set messages

Ziggurat currently does not allow clear the messages from the dead set, So we need a API which help clearing messages from the dead of the topic entity or channel queue.

Add In memory database support.

As we know Kafka stream consumers are bounded by Kafka partitions and making an I/O call would result in lag if Kafka throughput is very high in that case in memory database support would save the I/O time.

Dead set replay breaks when an application upgrades from Ziggurat 2 to 3

It was reported that dead_set messages pushed by 2.x version of Ziggurat can not be re-played after upgrading to 3.x.

Example:

Dead set message processed by Ziggurat 2.12.1:

{:foo "bar"}

Dead set message processed by Ziggurat 3.1.0:

{:message {:foo "bar"} :topic-entity "default" :retry-count nil}

Root Cause:
While pushing the message to a dead-set queue, :retry-count is removed from the message. When this message is fetched the dead set queue, convert-and-ack-message, while constructing a MessagePayload object, fills in the value for :retry-count from the message. This value is nil in the dead-set message.

Subsequently, the validation of this MessagePayload fails which forces the next consumer to again convert this message into a MessagePayload, resulting in mapper-function receiving a MessagePayload in place of the original message.

Recommend Projects

  • React photo React

    A declarative, efficient, and flexible JavaScript library for building user interfaces.

  • Vue.js photo Vue.js

    🖖 Vue.js is a progressive, incrementally-adoptable JavaScript framework for building UI on the web.

  • Typescript photo Typescript

    TypeScript is a superset of JavaScript that compiles to clean JavaScript output.

  • TensorFlow photo TensorFlow

    An Open Source Machine Learning Framework for Everyone

  • Django photo Django

    The Web framework for perfectionists with deadlines.

  • D3 photo D3

    Bring data to life with SVG, Canvas and HTML. 📊📈🎉

Recommend Topics

  • javascript

    JavaScript (JS) is a lightweight interpreted programming language with first-class functions.

  • web

    Some thing interesting about web. New door for the world.

  • server

    A server is a program made to process requests and deliver data to clients.

  • Machine learning

    Machine learning is a way of modeling and interpreting data that allows a piece of software to respond intelligently.

  • Game

    Some thing interesting about game, make everyone happy.

Recommend Org

  • Facebook photo Facebook

    We are working to build community through open source technology. NB: members must have two-factor auth.

  • Microsoft photo Microsoft

    Open source projects and samples from Microsoft.

  • Google photo Google

    Google ❤️ Open Source for everyone.

  • D3 photo D3

    Data-Driven Documents codes.