Gleam is a high performance and efficient distributed execution system, and also simple, generic, flexible and easy to customize.
Gleam is built in Go, and the user defined computation can be written in Go, Lua, Unix pipe tools, or any streaming programs.
It is convenient to write logic in Lua, but Lua is optional. Go is also supported with a little bit extra effort.
- Pure Go mappers and reducers have high performance and concurrency.
- Optional LuaJIT also has high performance comparable to C, Java, Go. It streamingly processes data, without context switch between Go and Lua.
- Data flows through memory, optionally to disk.
- Multiple map reduce steps are merged together for better performance.
- Gleam does not have the common GC problem that plagued other languages. Each executor is run in a separated OS process. The memory is managed by the OS. One machine can host many more executors.
- Gleam master and agent servers are memory efficient, consuming about 10 MB memory.
- Gleam tries to automatically adjust the required memory size based on data size hints, avoiding the try-and-error manual memory tuning effort.
- The Gleam flow can run standalone or distributed.
- Adjustable in memory mode or OnDisk mode.
- The Go code is much simpler to read than Scala, Java, C++.
- Optional LuaJIT FFI library can easily invoke any C functions, for even more performance or use any existing C libraries.
- (future) Write SQL with UDF written in Lua.
Gleam code defines the flow, specifying each dataset(vertex) and computation step(edge), and build up a directed acyclic graph(DAG). There are multiple ways to execute the DAG.
The default way is to run locally. This works in most cases.
Here we mostly talk about the distributed mode.
The distributed mode has several names to explain: Master, Agent, Executor, Driver.
- Driver is the program users write, it defines the flow, and talks to Master, Agents, and Executors.
- The Master is one single server that collects resource information from Agents.
- It stores transient resource information and can be restarted.
- When the Driver program starts, it asks the Master for available Exeutors on Agents.
- Agents runs on any machine that can run computations.
- Agents periodically send resource usage updates to Master.
- When the Driver program has executors assigned, it talks to the Agents to start Executors.
- Agents also manage datasets generated by each Executors.
- Executors are started by Agents. It will read inputs from external or previous datasets, process them, and output to a new dataset.
- The datasets are managed by Agents. By default, the data run only through memory and network, not touching slow disk.
- Optionally the data can be persist to disk.
By leaving it in memory, the flow can have back pressure, and can support stream computation naturally.
- [Gleam Wiki] (https://github.com/chrislusf/gleam/wiki)
- Installation
- Gleam Flow API GoDoc
- gleam-dev on Slack
Basically, you need to register the Go functions first. It will return a mapper or reducer function id, which we can pass it to the flow.
package main
import (
"strings"
"os"
"github.com/chrislusf/gleam/flow"
"github.com/chrislusf/gleam/gio"
)
var (
MapperTokenizer = gio.RegisterMapper(tokenize)
MapperAddOne = gio.RegisterMapper(addOne)
ReducerSum = gio.RegisterReducer(sum)
)
func main(){
gio.Init() // required, place it right after main() starts
flow.New().TextFile("/etc/passwd").
Mapper(MapperTokenizer). // invoke the registered "tokenize" mapper function.
Mapper(MapperAddOne). // invoke the registered "addOne" mapper function.
ReducerBy(ReducerSum). // invoke the registered "sum" reducer function.
Sort(flow.OrderBy(2, true)).
Fprintf(os.Stdout, "%s %d\n").Run()
}
func tokenize(row []interface{}) error {
line := string(row[0].([]byte))
for _, s := range strings.FieldsFunc(line, func(r rune) bool {
return !('A' <= r && r <= 'Z' || 'a' <= r && r <= 'z' || '0' <= r && r <= '9')
}) {
gio.Emit(s)
}
return nil
}
func addOne(row []interface{}) error {
word := string(row[0].([]byte))
gio.Emit(word, 1)
return nil
}
func sum(x, y interface{}) (interface{}, error) {
return x.(uint64) + y.(uint64), nil
}
A more blown up example is here. https://github.com/chrislusf/gleam/blob/master/examples/word_count_in_go/word_count_in_go.go
LuaJIT can greatly simplify the code. The full source code, not snippet, for word count:
package main
import (
"os"
"github.com/chrislusf/gleam/flow"
)
func main() {
flow.New().TextFile("/etc/passwd").FlatMap(`
function(line)
return line:gmatch("%w+")
end
`).Map(`
function(word)
return word, 1
end
`).ReduceBy(`
function(x, y)
return x + y
end
`).Fprintf(os.Stdout, "%s,%d\n").Run()
}
Another way to do the similar:
package main
import (
"os"
"github.com/chrislusf/gleam/flow"
)
func main() {
flow.New().TextFile("/etc/passwd").FlatMap(`
function(line)
return line:gmatch("%w+")
end
`).Pipe("sort").Pipe("uniq -c").Fprintf(os.Stdout, "%s\n").Run()
}
Assume there are file "a.csv" has fields "a1, a2, a3, a4, a5" and file "b.csv" has fields "b1, b2, b3". We want to join the rows where a1 = b2. And the output format should be "a1, a4, b3".
package main
import (
"os"
. "github.com/chrislusf/gleam/flow"
"github.com/chrislusf/gleam/plugins/csv"
)
func main() {
f := New()
a := f.ReadFile(csv.New("a.csv")).Select(Field(1,4)) // a1, a4
b := f.ReadFile(csv.New("b.csv")).Select(Field(2,3)) // b2, b3
a.Join(b).Fprintf(os.Stdout, "%s,%s,%s\n").Run() // a1, a4, b3
}
Unix Pipes are easy for sequential pipes, but limited to fan out, and even more limited to fan in.
With Gleam, fan-in and fan-out parallel pipes become very easy.
This example get a list of file names, partitioned into 3 groups, and then process them in parallel.
// word_count.go
package main
import (
"log"
"os"
"path/filepath"
"github.com/chrislusf/gleam/flow"
)
func main() {
fileNames, err := filepath.Glob("/Users/chris/Downloads/txt/en/ep-08-*.txt")
if err != nil {
log.Fatal(err)
}
flow.New().Strings(fileNames).Partition(3).PipeAsArgs("cat $1").FlatMap(`
function(line)
return line:gmatch("%w+")
end
`).Map(`
function(word)
return word, 1
end
`).ReduceBy(`
function(x, y)
return x + y
end
`).Fprintf(os.Stdout, "%s\t%d").Run()
}
Start a gleam master and several gleam agents
// start "gleam master" on a server
> go get github.com/chrislusf/gleam/distributed/gleam
> gleam master --address=":45326"
// start up "gleam agent" on some different servers or ports
// if a different server, remember to install Luajit and copy the MessagePack.lua file also.
> gleam agent --dir=2 --port 45327 --host=127.0.0.1
> gleam agent --dir=3 --port 45328 --host=127.0.0.1
Start a gleam master and several gleam agents
kubectl apply -f k8s/
After the flow is defined, the Run() function can be executed in different ways: local mode, distributed mode, or planner mode.
f := flow.New()
...
// local mode
f.Run()
// distributed mode
import "github.com/chrislusf/gleam/distributed"
f.Run(distributed.Option())
f.Run(distributed.Option().SetMaster("master_ip:45326"))
// distributed planner mode to print out logic plan
import "github.com/chrislusf/gleam/distributed"
f.Run(distributed.Planner())
LuaJIT is easy, but sometimes we really need to write in Go. It is a bit more complicated, but not much. Gleam allows us to write a simple Go code with mapper or reducer logic, and automatically send it over to Gleam agents to execute. See https://github.com/chrislusf/gleam/wiki/Write-Mapper-Reducer-in-Go
- Fault tolerant OnDisk().
- Support Cassandra Fast Data Extraction.
- Read data from Local, HDFS, or S3.
Gleam is just beginning. Here are a few todo items. Welcome any help!
- Refactor the plugin system to read external data.
- Add schema support for each dataset.
- Support using SQL as a flow step, similar to LINQ.
- Add windowing functions similar to google dataflow.
- Add dataset metadata for better caching of often re-calculated data.
Especially Need Help Now:
- Go implementation to read Parquet files.
Please start to use it and give feedback. Help is needed. Anything is welcome. Small things count: fix documentation, adding a logo, adding docker image, blog about it, share it, etc.
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.