Skip to content
This repository has been archived by the owner on Dec 1, 2018. It is now read-only.

Commit

Permalink
pick up the common part of elasticsearch sink
Browse files Browse the repository at this point in the history
  • Loading branch information
huangyuqi committed May 16, 2016
1 parent 48dfce4 commit a775491
Show file tree
Hide file tree
Showing 6 changed files with 169 additions and 202 deletions.
118 changes: 118 additions & 0 deletions common/elasticsearch/elasticsearch.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
// Copyright 2015 Google Inc. All Rights Reserved.
//
// 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.
package elasticsearch

import (
"fmt"
"net/url"

"github.com/golang/glog"
"github.com/olivere/elastic"
"github.com/pborman/uuid"
)

const (
ESIndex = "heapster"
)

// SaveDataFunc is a pluggable function to enforce limits on the object
type SaveDataFunc func(esClient *elastic.Client, indexName string, typeName string, sinkData interface{}) error

type ElasticSearchConfig struct {
EsClient *elastic.Client
Index string
NeedAuthen bool
EsUserName string
EsUserSecret string
EsNodes []string
}

// SaveDataIntoES save metrics and events to ES by using ES client
func SaveDataIntoES(esClient *elastic.Client, indexName string, typeName string, sinkData interface{}) error {
if indexName == "" || typeName == "" || sinkData == nil {
return nil
}
// Use the IndexExists service to check if a specified index exists.
exists, err := esClient.IndexExists(indexName).Do()
if err != nil {
return err
}
if !exists {
// Create a new index.
createIndex, err := esClient.CreateIndex(indexName).Do()
if err != nil {
return err
}
if !createIndex.Acknowledged {
return fmt.Errorf("failed to create Index in ES cluster: %s", err)
}
}
indexID := uuid.NewUUID()
_, err = esClient.Index().
Index(indexName).
Type(typeName).
Id(string(indexID)).
BodyJson(sinkData).
Do()
if err != nil {
return err
}
return nil
}

func CreateElasticSearchConfig(uri *url.URL) (*ElasticSearchConfig, error) {

var esConfig ElasticSearchConfig
opts, err := url.ParseQuery(uri.RawQuery)
if err != nil {
return nil, fmt.Errorf("failed to parser url's query string: %s", err)
}

// set the index for es,the default value is "heapster"
esConfig.Index = ESIndex
if len(opts["index"]) > 0 {
esConfig.Index = opts["index"][0]
}

// If the ES cluster needs authentication, the username and secret
// should be set in sink config.Else, set the Authenticate flag to false
esConfig.NeedAuthen = false
if len(opts["esUserName"]) > 0 && len(opts["esUserSecret"]) > 0 {
esConfig.EsUserName = opts["esUserName"][0]
esConfig.NeedAuthen = true
}

// set the URL endpoints of the ES's nodes. Notice that
// when sniffing is enabled, these URLs are used to initially sniff the
// cluster on startup.
if len(opts["nodes"]) < 1 {
return nil, fmt.Errorf("There is no node assigned for connecting ES cluster")
}
esConfig.EsNodes = append(esConfig.EsNodes, opts["nodes"]...)
glog.V(2).Infof("configing elasticsearch sink with ES's nodes - %v", esConfig.EsNodes)

var client *(elastic.Client)
if esConfig.NeedAuthen == false {
client, err = elastic.NewClient(elastic.SetURL(esConfig.EsNodes...))
} else {
client, err = elastic.NewClient(elastic.SetBasicAuth(esConfig.EsUserName, esConfig.EsUserSecret), elastic.SetURL(esConfig.EsNodes...))
}

if err != nil {
return nil, fmt.Errorf("failed to create ElasticSearch client: %v", err)
}
esConfig.EsClient = client
glog.V(2).Infof("elasticsearch sink configure successfully")
return &esConfig, nil
}
7 changes: 3 additions & 4 deletions docs/sink-configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -163,8 +163,7 @@ Thus, we can set `ES_SERVER_URL` to a dummy value, and provide nodes list or pro
in url's query string.
Besides, the following options can be set in query string:

* `timeseriesIndex` - ES's index for timeseries. Default: `heapster-metrics`
* `eventsIndex` - ES's index for events. Default: `heapster-events`
* `Index` - ES's index for metrics&events. Default: `heapster`

if the ES cluster needs authentication, we should provide the username and secret.

Expand All @@ -173,11 +172,11 @@ Besides, the following options can be set in query string:

Like this:

--sink="elasticsearch:?nodes=0.0.0.0:9200&timeseriesIndex=testMetric"
--sink="elasticsearch:?nodes=0.0.0.0:9200&Index=testMetric"

or

--sink="elasticsearch:?nodes=0.0.0.0:9200&eventsIndex=testEvent"
--sink="elasticsearch:?nodes=0.0.0.0:9200&Index=testEvent"

## Using multiple sinks

Expand Down
97 changes: 11 additions & 86 deletions events/sinks/elasticsearch/driver.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,36 +15,30 @@
package elasticsearch

import (
"fmt"
"net/url"
"sync"
"time"

"encoding/json"
"github.com/golang/glog"
"github.com/olivere/elastic"
"github.com/pborman/uuid"
esCommon "k8s.io/heapster/common/elasticsearch"
event_core "k8s.io/heapster/events/core"
"k8s.io/heapster/metrics/core"
kube_api "k8s.io/kubernetes/pkg/api"
)

const (
eventSeriesIndex = "heapster-metrics"
typeName = "k8s-heapster"
typeName = "events"
)

// LimitFunc is a pluggable function to enforce limits on the object
type SaveDataFunc func(esClient *elastic.Client, indexName string, typeName string, sinkData interface{}) error

type elasticSearchSink struct {
esClient *elastic.Client
saveDataFunc SaveDataFunc
eventSeriesIndex string
needAuthen bool
esUserName string
esUserSecret string
esNodes []string
esClient *elastic.Client
saveDataFunc SaveDataFunc
esConfig esCommon.ElasticSearchConfig
sync.RWMutex
}

Expand All @@ -69,7 +63,6 @@ func eventToPoint(event *kube_api.Event) (*EsSinkPoint, error) {
if err != nil {
return nil, err
}

point := EsSinkPoint{
EventTimestamp: event.LastTimestamp.Time.UTC(),
EventValue: value,
Expand All @@ -88,47 +81,13 @@ func eventToPoint(event *kube_api.Event) (*EsSinkPoint, error) {
func (sink *elasticSearchSink) ExportEvents(eventBatch *event_core.EventBatch) {
sink.Lock()
defer sink.Unlock()

for _, event := range eventBatch.Events {
point, err := eventToPoint(event)
if err != nil {
glog.Warningf("Failed to convert event to point: %v", err)
}
sink.saveDataFunc(sink.esClient, sink.eventSeriesIndex, typeName, point)
}
}

// SaveDataIntoES save metrics and events to ES by using ES client
func SaveDataIntoES(esClient *elastic.Client, indexName string, typeName string, sinkData interface{}) error {
if indexName == "" || typeName == "" || sinkData == nil {
return nil
}
// Use the IndexExists service to check if a specified index exists.
exists, err := esClient.IndexExists(indexName).Do()
if err != nil {
return err
}
if !exists {
// Create a new index.
createIndex, err := esClient.CreateIndex(indexName).Do()
if err != nil {
return err
}
if !createIndex.Acknowledged {
return fmt.Errorf("failed to create Index in ES cluster: %s", err)
}
}
indexID := uuid.NewUUID()
_, err = esClient.Index().
Index(indexName).
Type(typeName).
Id(string(indexID)).
BodyJson(sinkData).
Do()
if err != nil {
return err
sink.saveDataFunc(sink.esClient, sink.esConfig.Index, typeName, point)
}
return nil
}

func (sink *elasticSearchSink) Name() string {
Expand All @@ -140,49 +99,15 @@ func (sink *elasticSearchSink) Stop() {
}

func NewElasticSearchSink(uri *url.URL) (event_core.EventSink, error) {

var esSink elasticSearchSink
opts, err := url.ParseQuery(uri.RawQuery)
elasticsearchConfig, err := esCommon.CreateElasticSearchConfig(uri)
if err != nil {
return nil, fmt.Errorf("failed to parser url's query string: %s", err)
}

//set the index for eventSeries,the default value is "eventIndex"
esSink.eventSeriesIndex = eventSeriesIndex
if len(opts["eventSeriesIndex"]) > 0 {
esSink.eventSeriesIndex = opts["eventSeriesIndex"][0]
}

//If the ES cluster needs authentication, the username and secret
//should be set in sink config.Else, set the Authenticate flag to false
esSink.needAuthen = false
if len(opts["esUserName"]) > 0 && len(opts["esUserSecret"]) > 0 {
esSink.eventSeriesIndex = opts["esUserName"][0]
esSink.needAuthen = true
}

//set the URL endpoints of the ES's nodes. Notice that
// when sniffing is enabled, these URLs are used to initially sniff the
// cluster on startup.
if len(opts["nodes"]) < 1 {
return nil, fmt.Errorf("There is no node assigned for connecting ES cluster")
}
esSink.esNodes = append(esSink.esNodes, opts["nodes"]...)
glog.V(2).Infof("initializing elasticsearch sink with ES's nodes - %v", esSink.esNodes)

var client *(elastic.Client)
if esSink.needAuthen == false {
client, err = elastic.NewClient(elastic.SetURL(esSink.esNodes...))
} else {
client, err = elastic.NewClient(elastic.SetBasicAuth(esSink.esUserName, esSink.esUserSecret), elastic.SetURL(esSink.esNodes...))
}
glog.V(2).Infof("failed to config elasticsearch")
return nil, err

if err != nil {
return nil, fmt.Errorf("failed to create ElasticSearch client: %v", err)
}
esSink.esClient = client
esSink.saveDataFunc = SaveDataIntoES

esSink.esConfig = *elasticsearchConfig
esSink.saveDataFunc = esCommon.SaveDataIntoES
glog.V(2).Infof("elasticsearch sink setup successfully")
return &esSink, nil
}
27 changes: 11 additions & 16 deletions events/sinks/elasticsearch/driver_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"fmt"
"github.com/olivere/elastic"
"github.com/stretchr/testify/assert"
esCommon "k8s.io/heapster/common/elasticsearch"
"k8s.io/heapster/events/core"
kube_api "k8s.io/kubernetes/pkg/api"
kube_api_unversioned "k8s.io/kubernetes/pkg/api/unversioned"
Expand Down Expand Up @@ -53,13 +54,15 @@ func NewFakeSink() fakeESSink {
savedData := make([]dataSavedToES, 0)
return fakeESSink{
&elasticSearchSink{
esClient: &ESClient,
saveDataFunc: SaveDataIntoES_Stub,
eventSeriesIndex: "heapster-metric-index",
needAuthen: false,
esUserName: "admin",
esUserSecret: "admin",
esNodes: fakeSinkESNodes,
esClient: &ESClient,
saveDataFunc: SaveDataIntoES_Stub,
esConfig: esCommon.ElasticSearchConfig{
Index: "heapster-metric-index",
NeedAuthen: false,
EsUserName: "admin",
EsUserSecret: "admin",
EsNodes: fakeSinkESNodes,
},
},
savedData,
}
Expand All @@ -75,36 +78,28 @@ func TestStoreDataEmptyInput(t *testing.T) {
func TestStoreMultipleDataInput(t *testing.T) {
fakeSink := NewFakeSink()
timestamp := time.Now()

now := time.Now()
event1 := kube_api.Event{
Message: "event1",
Count: 100,
LastTimestamp: kube_api_unversioned.NewTime(now),
FirstTimestamp: kube_api_unversioned.NewTime(now),
}

event2 := kube_api.Event{
Message: "event2",
Count: 101,
LastTimestamp: kube_api_unversioned.NewTime(now),
FirstTimestamp: kube_api_unversioned.NewTime(now),
}

data := core.EventBatch{
Timestamp: timestamp,
Events: []*kube_api.Event{
&event1,
&event2,
},
}

fakeSink.ExportEvents(&data)

//expect msg string
// expect msg string
assert.Equal(t, 2, len(FakeESSink.savedData))
//msgsString := fmt.Sprintf("%s", FakeESSink.savedData)
//assert.Contains(t, msgsString, "")
fmt.Println(FakeESSink.savedData)
FakeESSink = fakeESSink{}
}

0 comments on commit a775491

Please sign in to comment.