You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@yunikorn.apache.org by GitBox <gi...@apache.org> on 2020/03/25 17:06:33 UTC

[GitHub] [incubator-yunikorn-core] adamantal opened a new pull request #109: YUNIKORN-14. Add rest API to retrieve app/container history info

adamantal opened a new pull request #109: YUNIKORN-14. Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109
 
 
   This PR only handles the server (core) part of this issue. The UI related changes will be handled in [YUNIKORN-8](https://issues.apache.org/jira/browse/YUNIKORN-8).
   
   The initial implementation consists of creating a `HistoricalPartitionInfo` object which stores some metrics (currently the number of application and containers). This is updated in 10 minutes by `HistoricalPartitionInfoUpdater`
   
   If there's need the poll time can be configurable, right now it is just hardcoded to 10 minutes, and we will keep 1 day of history (144 items).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400774721
 
 

 ##########
 File path: pkg/metrics/metrics_collector.go
 ##########
 @@ -0,0 +1,89 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"time"
+
+	dto "github.com/prometheus/client_model/go"
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+var tickerDefault = 1 * time.Minute
+
+type InternalMetricsCollector struct {
+	ticker         *time.Ticker
+	stopped        chan bool
+	metricsHistory *history.InternalMetricsHistory
+}
+
+func NewInternalMetricsCollector(hcInfo *history.InternalMetricsHistory) *InternalMetricsCollector {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &InternalMetricsCollector{
+		ticker,
+		finished,
+		hcInfo,
+	}
+}
+
+func (u *InternalMetricsCollector) StartService() {
+	go func() {
+		for {
+			select {
+			case <-u.stopped:
+				return
+			case <-u.ticker.C:
+				log.Logger().Debug("Adding current status to historical partition data")
+
+				totalAppsRunningMetric := &dto.Metric{}
+				totalAppsRunningMetricGauge := m.scheduler.getTotalApplicationsRunning()
+				err := (*totalAppsRunningMetricGauge).Write(totalAppsRunningMetric)
+				if err != nil {
+					log.Logger().Warn("Could not encode metric.", zap.Error(err))
+					continue
+				}
+
+				totalContainersRunningMetric := &dto.Metric{}
+				totalContainersRunningMetricCounter := m.scheduler.getAllocatedContainers()
+				err = (*totalContainersRunningMetricCounter).Write(totalContainersRunningMetric)
+				if err != nil {
+					log.Logger().Warn("Could not encode metric.", zap.Error(err))
+					continue
+				}
+
+				u.metricsHistory.Store(
+					int(*totalAppsRunningMetric.Gauge.Value),
+					int(*totalContainersRunningMetric.Counter.Value))
+			}
+		}
+	}()
+}
+
+func (u *InternalMetricsCollector) Stop() {
+	u.stopped <- true
+}
+
+func setInternalMetricsCollectorTickerForTest(newDefault time.Duration) {
 
 Review comment:
   Use a comment like `visible for test only` or something like it
   We should not use _test_ in the function name unless it is really a test.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401537591
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,65 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"gotest.tools/assert"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTickerForTest(500 * time.Millisecond)
+	metricsCollector := NewInternalMetricsCollector(metricsHistory)
+	metricsCollector.StartService()
+
+	go func() {
+		metrics := GetSchedulerMetrics()
+		i := 0
+		for i < 3 {
+			metrics.IncTotalApplicationsRunning()
+			metrics.AddAllocatedContainers(2)
+			i += 1
+			time.Sleep(400 * time.Millisecond)
 
 Review comment:
   with a shorter sleep I am OK with the test and the wait we put in.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401521436
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ApplicationHistoryDAOInfo{
+			Timestamp:         record.Timestamp.UnixNano(),
+			TotalApplications: strconv.Itoa(record.TotalApplications),
 
 Review comment:
   What is the intended usage of the `go fmt` command?
   
   I have the following error message running it from the project's root:
   ```
   can't load package: package github.com/apache/incubator-yunikorn-core: unknown import path "github.com/apache/incubator-yunikorn-core": cannot find module providing package github.com/apache/incubator-yunikorn-core
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400628285
 
 

 ##########
 File path: pkg/entrypoint/entrypoint.go
 ##########
 @@ -57,6 +59,9 @@ func startAllServicesWithParameters(opts StartupOptions) *ServiceContext {
 	scheduler := scheduler.NewScheduler(cache)
 	proxy := rmproxy.NewRMProxy()
 
+	imHistory := history.NewInternalMetricsHistory(1440)
 
 Review comment:
   Add a new StartupOptions entry (int), set it to 0 for a manual scheduler and set it to a value for the normal startup case.
   We do not need it during testing in normal cases. If the int is 0 skip it like we do with the web app flag

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401524905
 
 

 ##########
 File path: pkg/entrypoint/entrypoint.go
 ##########
 @@ -73,14 +73,19 @@ func startAllServicesWithParameters(opts StartupOptions) *ServiceContext {
 	cache.StartService(eventHandler)
 	scheduler.StartService(eventHandler, opts.manualScheduleFlag)
 	proxy.StartService(eventHandler)
-	metricsCollector.StartService()
 
 	context := &ServiceContext{
 		RMProxy:   proxy,
 		Cache:     cache,
 		Scheduler: scheduler,
 	}
 
+	imHistory := history.NewInternalMetricsHistory(opts.metricsHistorySize)
+	if opts.metricsHistorySize != 0 {
 
 Review comment:
   Agreed. Fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401316780
 
 

 ##########
 File path: pkg/metrics/metrics_collector.go
 ##########
 @@ -0,0 +1,81 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"time"
+
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+var tickerDefault = 1 * time.Minute
+
+type InternalMetricsCollector struct {
 
 Review comment:
   Do we need to export?
   IDE says that it is not needed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-606431752
 
 
   Rebased due to conflict in the `go.mod` file (YUNIKORN-63).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-608116268
 
 
   Looks good to me, thanks @wilfred-s , @adamantal !
   I think we can merge this. Only one minor thing, we might want to check with the front-end. I saw when we are not able to get app/container count, it is set to `-1`.  We need to make sure the front-end supports this value.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402148077
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,49 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	if imHistory != nil {
+		var result []*dao.ApplicationHistoryDAOInfo
+		records := imHistory.GetRecords()
+		for _, record := range records {
+			if record != nil {
 
 Review comment:
   Fixed this, but I think we're relying too much on the actual implementation of the `internalMetricsHistory`, and it makes the code too fragile if it is modified in the future.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402095164
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,49 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	if imHistory != nil {
+		var result []*dao.ApplicationHistoryDAOInfo
+		records := imHistory.GetRecords()
+		for _, record := range records {
+			if record != nil {
+				element := &dao.ApplicationHistoryDAOInfo{
+					Timestamp:         record.Timestamp.UnixNano(),
+					TotalApplications: strconv.Itoa(record.TotalApplications),
+				}
+				result = append(result, element)
+			}
+		}
+		if err := json.NewEncoder(w).Encode(result); err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+		}
+	} else {
+		http.Error(w, "Internal metrics collection is not enabled.", http.StatusInternalServerError)
+	}
+}
+
+func GetContainerHistory(w http.ResponseWriter, r *http.Request) {
 
 Review comment:
   same comments as per `GetApplicationHistory`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400905277
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,74 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+	mutex   sync.Mutex
+}
+
+type MetricsRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
+	return &InternalMetricsHistory{
+		records: make([]*MetricsRecord, 0),
+		limit:   limit,
+	}
+}
+
+func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
+	h.mutex.Lock()
+	defer h.mutex.Unlock()
+
+	h.records = append(h.records,
 
 Review comment:
   Nice catch. I actually got through that post during my Go tutorial, but didn't realize that I made a mistake here until now.
   I started with a containers/list implementation, but got into problems, because it was not typesafe and had to case everywhere where it is used.
   
   Implemented a circular, sized array using the pointer and the limit variables. If there's any official typesafe implementation in Golang, or some popular package to carry out this, please let me know and I'll use that. I think this is the most light implementation that we could use.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s closed pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s closed pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398556155
 
 

 ##########
 File path: pkg/cache/history_collector.go
 ##########
 @@ -0,0 +1,76 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 cache
+
+import (
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/cache/history"
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+)
+
+var tickerDefault = 10 * time.Minute
+
+type HistoryProvider interface {
+	GetTotalApplicationCount() int
+	GetTotalAllocationCount() int
+}
+
+type HistoricalPartitionInfoUpdater struct {
+	finished chan bool
+	ticker   *time.Ticker
+	pInfo    HistoryProvider
+	hpInfo   *history.HistoricalPartitionInfo
+}
+
+func NewHistoricalPartitionInfoUpdater(pInfo HistoryProvider, hpInfo *history.HistoricalPartitionInfo) *HistoricalPartitionInfoUpdater {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &HistoricalPartitionInfoUpdater{
+		finished,
+		ticker,
+		pInfo,
+		hpInfo,
+	}
+}
+
+func (hpInfoUpdater *HistoricalPartitionInfoUpdater) Start() {
+	go func() {
+		for {
+			select {
+			case <-hpInfoUpdater.finished:
+				return
+			case <-hpInfoUpdater.ticker.C:
+				log.Logger().Info("Adding current status to historical partition data")
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400632930
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,74 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+	mutex   sync.Mutex
 
 Review comment:
   just use `sync.Mutex`, no need to use a var here. See `scheduling_application.go` as an example.
   Should use a RWMutex instead of a standard Mutex

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400776362
 
 

 ##########
 File path: pkg/metrics/init.go
 ##########
 @@ -54,6 +56,7 @@ type CoreSchedulerMetrics interface {
 	// Metrics Ops related to ScheduledAllocationSuccesses
 	IncAllocatedContainer()
 	AddAllocatedContainers(value int)
+	getAllocatedContainers() *prometheus.Counter
 
 Review comment:
   can we not simply return the primitive we need?
   Hide all the prometheus logic inside one file, see other review comments on how.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401538135
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,74 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+	mutex   sync.Mutex
+}
+
+type MetricsRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
+	return &InternalMetricsHistory{
+		records: make([]*MetricsRecord, 0),
+		limit:   limit,
+	}
+}
+
+func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
+	h.mutex.Lock()
+	defer h.mutex.Unlock()
+
+	h.records = append(h.records,
 
 Review comment:
   new impl looks good

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400919232
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,99 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+
+	// internal implementation of limited array
+	pointer int
+	full    bool
+
+	sync.RWMutex
+}
+
+type MetricsRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
+	return &InternalMetricsHistory{
+		records: make([]*MetricsRecord, limit),
+		limit:   limit,
+	}
+}
+
+func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
+	h.Lock()
+	defer h.Unlock()
+
+	h.records[h.pointer] = &MetricsRecord{
+		time.Now(),
+		totalApplications,
+		totalContainers,
+	}
+
+	h.pointer++
+	if h.pointer == h.limit {
+		h.pointer = 0
+		h.full = true
+	}
+}
+
+// the return of this function is ordered by the time of addition
+func (h *InternalMetricsHistory) GetRecords() []*MetricsRecord {
+	h.RLock()
+	defer h.RUnlock()
+
+	returnRecords := make([]*MetricsRecord, 0)
+	if h.full {
+		pointer := h.pointer
+		for i := 0; i < h.limit; i++ {
+			returnRecords = append(returnRecords, h.records[pointer])
+			pointer++
+			if pointer == h.limit {
+				pointer = 0
+			}
+		}
+	} else {
+		for _, record := range h.records {
+			if record != nil {
+				returnRecords = append(returnRecords, record)
+			}
+		}
+	}
+	return returnRecords
+}
+
+func (h *InternalMetricsHistory) GetLimit() int {
 
 Review comment:
   Limit is set only one time when system is booted up, could you please confirm on that.
   If that's the case, do we need a mutex here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401315998
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,99 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+
+	// internal implementation of limited array
+	pointer int
+	full    bool
+
+	sync.RWMutex
+}
+
+type MetricsRecord struct {
 
 Review comment:
   Do we need to export this?
   IDE says there is no need to export.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401283749
 
 

 ##########
 File path: pkg/metrics/scheduler.go
 ##########
 @@ -200,8 +201,10 @@ func (m *SchedulerMetrics) AddAllocatedContainers(value int) {
 	m.allocatedContainers.Add(float64(value))
 }
 
-func (m *SchedulerMetrics) getAllocatedContainers() *prometheus.Counter {
-	return &m.allocatedContainers
+func (m *SchedulerMetrics) getAllocatedContainers() (int, error) {
+	metric := &dto.Metric{}
+	err := m.allocatedContainers.Write(metric)
+	return int(*metric.Counter.Value), err
 
 Review comment:
   if `err` is set this will cause a nil pointer and thus a panic. We have to guard:
   ```
   	int value
   	if err := m.allocatedContainers.Write(metricDto); err == nil {
   		value = int(*metricDto.Counter.Value)
   	}
   	return value, err
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401287269
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,99 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+
+	// internal implementation of limited array
+	pointer int
+	full    bool
+
+	sync.RWMutex
+}
+
+type MetricsRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
+	return &InternalMetricsHistory{
+		records: make([]*MetricsRecord, limit),
+		limit:   limit,
+	}
+}
+
+func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
+	h.Lock()
+	defer h.Unlock()
+
+	h.records[h.pointer] = &MetricsRecord{
+		time.Now(),
+		totalApplications,
+		totalContainers,
+	}
+
+	h.pointer++
+	if h.pointer == h.limit {
+		h.pointer = 0
+		h.full = true
+	}
+}
+
+// the return of this function is ordered by the time of addition
+func (h *InternalMetricsHistory) GetRecords() []*MetricsRecord {
+	h.RLock()
+	defer h.RUnlock()
+
+	returnRecords := make([]*MetricsRecord, 0)
+	if h.full {
+		pointer := h.pointer
+		for i := 0; i < h.limit; i++ {
+			returnRecords = append(returnRecords, h.records[pointer])
+			pointer++
+			if pointer == h.limit {
+				pointer = 0
+			}
+		}
+	} else {
+		for _, record := range h.records {
+			if record != nil {
+				returnRecords = append(returnRecords, record)
+			}
+		}
+	}
+	return returnRecords
+}
+
+func (h *InternalMetricsHistory) GetLimit() int {
 
 Review comment:
   yes we do. In the `GetRecords` call we also access the same limit value and that call comes from a different go routine (webapp). If we allow unlocked access the code compiled with the `race` option could log data race issues.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400924104
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ApplicationHistoryDAOInfo{
+			Timestamp:         record.Timestamp.UnixNano(),
+			TotalApplications: strconv.Itoa(record.TotalApplications),
+		}
+		result = append(result, element)
+	}
+
+	if err := json.NewEncoder(w).Encode(result); err != nil {
+		http.Error(w, err.Error(), http.StatusInternalServerError)
+	}
+}
+
+func GetContainerHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ContainerHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ContainerHistoryDAOInfo{
+			Timestamp:       record.Timestamp.UnixNano(),
+			TotalContainers: strconv.Itoa(record.TotalContainers),
 
 Review comment:
   are we having only TotalContainers/Apps returned in an array when we query /ws/v1/history/containers ?
   Cud u pls share a sample response?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-608183420
 
 
   Thank you Weiwei, I'll log a jira against the webapp to make sure we can handle the -1 value.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400772063
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
 
 Review comment:
   Need to check the impact of the earlier mentioned unlimited growth solution on this iteration loop

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401323715
 
 

 ##########
 File path: go.mod
 ##########
 @@ -4,7 +4,7 @@ go 1.12
 
 require (
 	github.com/apache/incubator-yunikorn-scheduler-interface v0.0.0-20200327234544-149aaa3d8e48
-	github.com/golang/protobuf v1.3.5 // indirect
 
 Review comment:
   a `go mod tidy` adds it back again on my system so I think we need to follow that and remove this from the change

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402148539
 
 

 ##########
 File path: pkg/metrics/scheduler.go
 ##########
 @@ -200,6 +201,14 @@ func (m *SchedulerMetrics) AddAllocatedContainers(value int) {
 	m.allocatedContainers.Add(float64(value))
 }
 
+func (m *SchedulerMetrics) getAllocatedContainers() (value int, err error) {
 
 Review comment:
   Thanks, didn't know that. Is it standardised on a per file or a per package base?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400629485
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics_test.go
 ##########
 @@ -0,0 +1,55 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"testing"
+
+	"gotest.tools/assert"
+)
+
+func TestHistoricalClusterInfo(t *testing.T) {
+	limit := 2
+	hpInfo := NewInternalMetricsHistory(limit)
+
+	assert.Equal(t, limit, hpInfo.GetLimit(), "Limit should have been set to 2!")
+
+	hpInfo.Store(2, 3)
+	assert.Equal(t, 1, len(hpInfo.GetRecords()),
+		"Expected to have 1 record")
 
 Review comment:
   We don't have line length limits don't split them over multiple lines

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398555490
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
+
+type HistoricalPartitionInfo struct {
+	records []*PartitionInfoRecord
 
 Review comment:
   Added a simple `sync.Mutex` for this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402094932
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,49 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	if imHistory != nil {
 
 Review comment:
   use unindented flow instead of an else clause:
   ```
   if imHistory == nil {
   	http.Error...
   	return
   }
   var result ....
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398258503
 
 

 ##########
 File path: pkg/cache/partition_info.go
 ##########
 @@ -103,6 +106,11 @@ func newPartitionInfo(partition configs.PartitionConfig, rmID string, info *Clus
 	// TODO get the resolver from the config
 	p.userGroupCache = security.GetUserGroupCache("")
 
+	hpInfo := &history.HistoricalPartitionInfo{}
+	p.history = hpInfo
+	p.historyUpdater = NewHistoricalPartitionInfoUpdater(p, hpInfo)
+	p.historyUpdater.Start()
+
 
 Review comment:
   Can we move this to `cluster_info.StartService()` ?
   Having a Start call in the `newPartitionInfo()` seems not appropriate. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-606051669
 
 
   Created a brand new architecture according to @wilfred-s comment on the jira.
   
   Checked on my local with the `make run` command on k8shim, also starting 3 sleep pods using the `deployments/examples/sleep/sleeppods.yaml` file.
   Example output of the `(localhost:9080)/ws/v1/history/containers` endpoint:
   
   ```json
   [
       {
           "timestamp": 1585579879950325000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585579939947939000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585579999942027000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585580059941905000,
           "totalContainers": "3"
       }
   ]
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401303982
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -40,35 +45,55 @@ type MetricsRecord struct {
 
 func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
 	return &InternalMetricsHistory{
-		records: make([]*MetricsRecord, 0),
+		records: make([]*MetricsRecord, limit),
 		limit:   limit,
 	}
 }
 
 func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
-	h.mutex.Lock()
-	defer h.mutex.Unlock()
+	h.Lock()
+	defer h.Unlock()
+
+	h.records[h.pointer] = &MetricsRecord{
+		time.Now(),
+		totalApplications,
+		totalContainers,
+	}
 
-	h.records = append(h.records,
-		&MetricsRecord{
-			time.Now(),
-			totalApplications,
-			totalContainers,
-		})
-	if len(h.records) > h.limit {
-		// remove oldest entry
-		h.records = h.records[1:]
+	h.pointer++
+	if h.pointer == h.limit {
+		h.pointer = 0
+		h.full = true
 	}
 }
 
+// the return of this function is ordered by the time of addition
 func (h *InternalMetricsHistory) GetRecords() []*MetricsRecord {
-	h.mutex.Lock()
-	defer h.mutex.Unlock()
-	return h.records
+	h.RLock()
+	defer h.RUnlock()
+
+	returnRecords := make([]*MetricsRecord, 0)
+	if h.full {
+		pointer := h.pointer
+		for i := 0; i < h.limit; i++ {
+			returnRecords = append(returnRecords, h.records[pointer])
+			pointer++
+			if pointer == h.limit {
+				pointer = 0
+			}
+		}
+	} else {
+		for _, record := range h.records {
+			if record != nil {
+				returnRecords = append(returnRecords, record)
+			}
+		}
+	}
 
 Review comment:
   This can be done far simpler:
   ```
   	returnRecords := make([]*MetricsRecord, h.limit-h.pointer)
   	copy(returnRecords, h.records[h.pointer:])
   	returnRecords = append(returnRecords, h.records[:h.pointer]...)
   	return returnRecords
   ```
   No special casing for the full list, no ranging, less allocations.
   Handling nil records in the list build in the webapp is simpler than this if..else structure

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398252203
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
 
 Review comment:
   10 minutes will be too long. can we use a 1 minute interval, and keep 12 hours data?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401546638
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
 
 Review comment:
   Could you check whether the fixing commit is good? I hope I get your first item right.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398556585
 
 

 ##########
 File path: pkg/cache/partition_info.go
 ##########
 @@ -103,6 +106,11 @@ func newPartitionInfo(partition configs.PartitionConfig, rmID string, info *Clus
 	// TODO get the resolver from the config
 	p.userGroupCache = security.GetUserGroupCache("")
 
+	hpInfo := &history.HistoricalPartitionInfo{}
+	p.history = hpInfo
+	p.historyUpdater = NewHistoricalPartitionInfoUpdater(p, hpInfo)
+	p.historyUpdater.Start()
+
 
 Review comment:
   I only moved the `Start` call to `StartService`, but I kept the construction of the `HistoricalParitionInfoUpdater` object in the constructor.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398249521
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
+
+type HistoricalPartitionInfo struct {
+	records []*PartitionInfoRecord
+}
+
+type PartitionInfoRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func (hpInfo *HistoricalPartitionInfo) Store(totalApplications, totalContainers int) {
 
 Review comment:
   let's keep the ref short, `hpInfo` -> `h`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400874702
 
 

 ##########
 File path: pkg/metrics/init.go
 ##########
 @@ -54,6 +56,7 @@ type CoreSchedulerMetrics interface {
 	// Metrics Ops related to ScheduledAllocationSuccesses
 	IncAllocatedContainer()
 	AddAllocatedContainers(value int)
+	getAllocatedContainers() *prometheus.Counter
 
 Review comment:
   Makes sense, fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402097705
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,62 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTicker(5 * time.Millisecond)
+	metricsCollector := NewInternalMetricsCollector(metricsHistory)
+	metricsCollector.StartService()
+
+	go func() {
+		metrics := GetSchedulerMetrics()
+		i := 0
+		for i < 3 {
+			metrics.IncTotalApplicationsRunning()
+			metrics.AddAllocatedContainers(2)
+			i += 1
+			time.Sleep(4 * time.Millisecond)
+		}
+	}()
+
+	time.Sleep(11 * time.Millisecond)
+	metricsCollector.Stop()
+
+	records := metricsHistory.GetRecords()
+	assert.Equal(t, 3, len(records), "Expected exactly 2 history records")
+	for i, record := range records {
+		if i == 0 {
 
 Review comment:
   use a switch statement or just direct `records[0]` etc

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400921640
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ApplicationHistoryDAOInfo{
+			Timestamp:         record.Timestamp.UnixNano(),
+			TotalApplications: strconv.Itoa(record.TotalApplications),
 
 Review comment:
   in general, please check spacing. is it tab or spaces?
   code allignment?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-604515442
 
 
   Tested the patch locally with the following results:
   ```
   AAntal-MBP15:yunikorn-core adamantal$ kubectl logs yunikorn-scheduler-76ff98dff4-lb77q yunikorn-scheduler-k8s | grep "Adding current status"
   2020-03-26T15:35:10.911Z	DEBUG	cache/history_collector.go:61	Adding current status to historical partition data
   2020-03-26T15:36:10.912Z	DEBUG	cache/history_collector.go:61	Adding current status to historical partition data
   2020-03-26T15:37:10.912Z	DEBUG	cache/history_collector.go:61	Adding current status to historical partition data
   ```
   So as expected, we are collecting the metrics in every minute.
   
   I also checked the rest API endpoint, and it seems like there's a bug. I missed a "/" from the beginning of the REST endpoints, so they are not accessible. I will fix it, test it locally and update the PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402092630
 
 

 ##########
 File path: pkg/metrics/scheduler.go
 ##########
 @@ -265,6 +274,12 @@ func (m *SchedulerMetrics) SetTotalApplicationsRunning(value int) {
 	m.totalApplicationsRunning.Set(float64(value))
 }
 
+func (m *SchedulerMetrics) getTotalApplicationsRunning() (int, error) {
+	metric := &dto.Metric{}
+	err := m.totalApplicationsRunning.Write(metric)
+	return int(*metric.Gauge.Value), err
+}
 
 Review comment:
   See `getAllocatedContainers` same nil pointer here, solution is same, don't use named returns

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398556003
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
 
 Review comment:
   Modified the constants accordingly

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398555828
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401307027
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ApplicationHistoryDAOInfo{
+			Timestamp:         record.Timestamp.UnixNano(),
+			TotalApplications: strconv.Itoa(record.TotalApplications),
+		}
+		result = append(result, element)
+	}
+
+	if err := json.NewEncoder(w).Encode(result); err != nil {
+		http.Error(w, err.Error(), http.StatusInternalServerError)
+	}
+}
+
+func GetContainerHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ContainerHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ContainerHistoryDAOInfo{
+			Timestamp:       record.Timestamp.UnixNano(),
+			TotalContainers: strconv.Itoa(record.TotalContainers),
 
 Review comment:
   Documented in jira [YUNIKORN-8](https://issues.apache.org/jira/browse/YUNIKORN-8)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-606121347
 
 
   Thanks @adamantal !
   @wilfred-s , can you help to review this change?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398253371
 
 

 ##########
 File path: pkg/cache/history_collector.go
 ##########
 @@ -0,0 +1,76 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 cache
+
+import (
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/cache/history"
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+)
+
+var tickerDefault = 10 * time.Minute
+
+type HistoryProvider interface {
+	GetTotalApplicationCount() int
+	GetTotalAllocationCount() int
+}
+
+type HistoricalPartitionInfoUpdater struct {
+	finished chan bool
+	ticker   *time.Ticker
+	pInfo    HistoryProvider
+	hpInfo   *history.HistoricalPartitionInfo
+}
+
+func NewHistoricalPartitionInfoUpdater(pInfo HistoryProvider, hpInfo *history.HistoricalPartitionInfo) *HistoricalPartitionInfoUpdater {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &HistoricalPartitionInfoUpdater{
+		finished,
+		ticker,
+		pInfo,
+		hpInfo,
+	}
+}
+
+func (hpInfoUpdater *HistoricalPartitionInfoUpdater) Start() {
+	go func() {
+		for {
+			select {
+			case <-hpInfoUpdater.finished:
+				return
+			case <-hpInfoUpdater.ticker.C:
+				log.Logger().Info("Adding current status to historical partition data")
 
 Review comment:
   `INFO` to `DEBUG`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398556099
 
 

 ##########
 File path: pkg/cache/history_collector.go
 ##########
 @@ -0,0 +1,76 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 cache
+
+import (
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/cache/history"
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+)
+
+var tickerDefault = 10 * time.Minute
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398252947
 
 

 ##########
 File path: pkg/cache/history_collector.go
 ##########
 @@ -0,0 +1,76 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 cache
+
+import (
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/cache/history"
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+)
+
+var tickerDefault = 10 * time.Minute
 
 Review comment:
   change to 1 minute

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401526481
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,99 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+
+	// internal implementation of limited array
+	pointer int
+	full    bool
+
+	sync.RWMutex
+}
+
+type MetricsRecord struct {
 
 Review comment:
   It is used in `webservice/handlers.go`, though the name `MetricsRecord` is not explicitly used. I fix it then.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402147235
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,62 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTicker(5 * time.Millisecond)
+	metricsCollector := NewInternalMetricsCollector(metricsHistory)
+	metricsCollector.StartService()
+
+	go func() {
+		metrics := GetSchedulerMetrics()
+		i := 0
+		for i < 3 {
+			metrics.IncTotalApplicationsRunning()
+			metrics.AddAllocatedContainers(2)
+			i += 1
+			time.Sleep(4 * time.Millisecond)
+		}
+	}()
+
+	time.Sleep(11 * time.Millisecond)
+	metricsCollector.Stop()
+
+	records := metricsHistory.GetRecords()
+	assert.Equal(t, 3, len(records), "Expected exactly 2 history records")
 
 Review comment:
   Silly of me, thanks for reminding!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400773432
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,65 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"gotest.tools/assert"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTickerForTest(500 * time.Millisecond)
+	metricsCollector := NewInternalMetricsCollector(metricsHistory)
+	metricsCollector.StartService()
+
+	go func() {
+		metrics := GetSchedulerMetrics()
+		i := 0
+		for i < 3 {
+			metrics.IncTotalApplicationsRunning()
+			metrics.AddAllocatedContainers(2)
+			i += 1
+			time.Sleep(400 * time.Millisecond)
 
 Review comment:
   Do we really need to sleep between each update? Extends the test run unneeded

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398251737
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
 
 Review comment:
   can we add a construct method for this, with 1 parameter about the size?
   so we can remove `setLimitForTest()`, and just construct different objects with different passing size.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401304773
 
 

 ##########
 File path: pkg/entrypoint/entrypoint.go
 ##########
 @@ -73,14 +73,19 @@ func startAllServicesWithParameters(opts StartupOptions) *ServiceContext {
 	cache.StartService(eventHandler)
 	scheduler.StartService(eventHandler, opts.manualScheduleFlag)
 	proxy.StartService(eventHandler)
-	metricsCollector.StartService()
 
 	context := &ServiceContext{
 		RMProxy:   proxy,
 		Cache:     cache,
 		Scheduler: scheduler,
 	}
 
+	imHistory := history.NewInternalMetricsHistory(opts.metricsHistorySize)
+	if opts.metricsHistorySize != 0 {
 
 Review comment:
   if the metrics to keep == 0 we should not create a `NewInternalMetricsHistory` at all.
   We should not start the collector and handle a `nil` history in the webapp. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402148774
 
 

 ##########
 File path: pkg/metrics/metrics_collector.go
 ##########
 @@ -0,0 +1,81 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"time"
+
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+var tickerDefault = 1 * time.Minute
+
+type internalMetricsCollector struct {
+	ticker         *time.Ticker
+	stopped        chan bool
+	metricsHistory *history.InternalMetricsHistory
+}
+
+func NewInternalMetricsCollector(hcInfo *history.InternalMetricsHistory) *internalMetricsCollector {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &internalMetricsCollector{
+		ticker,
+		finished,
+		hcInfo,
+	}
+}
+
+func (u *internalMetricsCollector) StartService() {
+	go func() {
+		for {
+			select {
+			case <-u.stopped:
+				return
+			case <-u.ticker.C:
+				log.Logger().Debug("Adding current status to historical partition data")
+
+				totalAppsRunning, err := m.scheduler.getTotalApplicationsRunning()
+				if err != nil {
+					log.Logger().Warn("Could not encode totalApplications metric.", zap.Error(err))
+					continue
 
 Review comment:
   I used -1, added a comment on the class.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400918054
 
 

 ##########
 File path: go.mod
 ##########
 @@ -4,7 +4,7 @@ go 1.12
 
 require (
 	github.com/apache/incubator-yunikorn-scheduler-interface v0.0.0-20200327234544-149aaa3d8e48
-	github.com/golang/protobuf v1.3.5 // indirect
 
 Review comment:
   could we avoid this change

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
sunilgovind commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400919868
 
 

 ##########
 File path: pkg/metrics/metrics_collector.go
 ##########
 @@ -0,0 +1,81 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"time"
+
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+var tickerDefault = 1 * time.Minute
+
+type InternalMetricsCollector struct {
+	ticker         *time.Ticker
+	stopped        chan bool
+	metricsHistory *history.InternalMetricsHistory
+}
+
+func NewInternalMetricsCollector(hcInfo *history.InternalMetricsHistory) *InternalMetricsCollector {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &InternalMetricsCollector{
+		ticker,
+		finished,
+		hcInfo,
+	}
+}
+
+func (u *InternalMetricsCollector) StartService() {
+	go func() {
+		for {
+			select {
+			case <-u.stopped:
+				return
+			case <-u.ticker.C:
+				log.Logger().Debug("Adding current status to historical partition data")
+
+				totalAppsRunning, err := m.scheduler.getTotalApplicationsRunning()
+				if err != nil {
+					log.Logger().Warn("Could not encode metric.", zap.Error(err))
+					continue
+				}
+				totalContainersRunning, err := m.scheduler.getAllocatedContainers()
+				if err != nil {
+					log.Logger().Warn("Could not encode metric.", zap.Error(err))
 
 Review comment:
   cud we add metric name also here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400733175
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -0,0 +1,74 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"sync"
+	"time"
+)
+
+// This class collects basic information about the cluster
+// for the web UI's front page.
+// For more detailed metrics collection use Prometheus.
+type InternalMetricsHistory struct {
+	records []*MetricsRecord
+	limit   int
+	mutex   sync.Mutex
+}
+
+type MetricsRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
+	return &InternalMetricsHistory{
+		records: make([]*MetricsRecord, 0),
+		limit:   limit,
+	}
+}
+
+func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
+	h.mutex.Lock()
+	defer h.mutex.Unlock()
+
+	h.records = append(h.records,
 
 Review comment:
   This is dangerous. We grow the slice and the underlying array . Since we keep this reference around for the lifetime of the scheduler we grow this unlimited. The limit specified is thus not really a limit of the array just of the reachable part of it.
   This is a long lived object we can not do it this way. Check the section [slice internals](https://blog.golang.org/slices-intro) of this post. We will need a copy or move to make sure we release the underlying array and not grow without bounds.
   I overlooked that in the last review round.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400773852
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,65 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"gotest.tools/assert"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTickerForTest(500 * time.Millisecond)
 
 Review comment:
   Can we shorten this to something like 10 msec?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398556921
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -235,3 +236,57 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
 
 Review comment:
   Agreed. Modified this and other classes in the PR accordingly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402092153
 
 

 ##########
 File path: pkg/metrics/scheduler.go
 ##########
 @@ -200,6 +201,14 @@ func (m *SchedulerMetrics) AddAllocatedContainers(value int) {
 	m.allocatedContainers.Add(float64(value))
 }
 
+func (m *SchedulerMetrics) getAllocatedContainers() (value int, err error) {
 
 Review comment:
   We should not mix named and unnamed returns.
   We have standardised on unnamed everywhere else.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402093818
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,49 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	if imHistory != nil {
+		var result []*dao.ApplicationHistoryDAOInfo
+		records := imHistory.GetRecords()
+		for _, record := range records {
+			if record != nil {
 
 Review comment:
   Can we have records that are not nil after the first nil record? I think not, but please check.
   If we cannot why not use a break to jump out of the loop:
   ```
   if record == nil {
   	break
   }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401308153
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
 
 Review comment:
   Two issues:
   * if kept records is 0 we should have a response that shows it is not tracked. Also see my comment around creating an imHistory in that case and handling nil.
   * during the startup period we have nil records in the list until the point in time that we have fully filled the array. We need to be able to handle nil records and skip over them otherwise we will have panics.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402097912
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,62 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTicker(5 * time.Millisecond)
+	metricsCollector := NewInternalMetricsCollector(metricsHistory)
+	metricsCollector.StartService()
+
+	go func() {
+		metrics := GetSchedulerMetrics()
+		i := 0
+		for i < 3 {
+			metrics.IncTotalApplicationsRunning()
+			metrics.AddAllocatedContainers(2)
+			i += 1
+			time.Sleep(4 * time.Millisecond)
+		}
+	}()
+
+	time.Sleep(11 * time.Millisecond)
+	metricsCollector.Stop()
+
+	records := metricsHistory.GetRecords()
+	assert.Equal(t, 3, len(records), "Expected exactly 2 history records")
 
 Review comment:
   nit: text not updated to 3

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398270209
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -235,3 +236,57 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	lists := gClusterInfo.ListPartitions()
+	for _, k := range lists {
 
 Review comment:
   right now it reports back per partition
   I don't think front-end expect this. we should aggregate this to the cluster level. Can we change the `HistoricPartitionInfo` to `HistoricClusterInfo`? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400875481
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,65 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"gotest.tools/assert"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+func TestHistoricalPartitionInfoUpdater(t *testing.T) {
+	metricsHistory := history.NewInternalMetricsHistory(3)
+	setInternalMetricsCollectorTickerForTest(500 * time.Millisecond)
+	metricsCollector := NewInternalMetricsCollector(metricsHistory)
+	metricsCollector.StartService()
+
+	go func() {
+		metrics := GetSchedulerMetrics()
+		i := 0
+		for i < 3 {
+			metrics.IncTotalApplicationsRunning()
+			metrics.AddAllocatedContainers(2)
+			i += 1
+			time.Sleep(400 * time.Millisecond)
 
 Review comment:
   I wanted to wait for at least 2 cycles. Do you think it is still an issue after ticker has been configured to 10msec (from comment below)?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
yangwwei commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398248259
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
+
+type HistoricalPartitionInfo struct {
+	records []*PartitionInfoRecord
 
 Review comment:
   you will need a lock for this.
   As you might update the records and the same time the rest server is retrieving the values.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402091162
 
 

 ##########
 File path: pkg/metrics/metrics_collector.go
 ##########
 @@ -0,0 +1,81 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"time"
+
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+var tickerDefault = 1 * time.Minute
+
+type internalMetricsCollector struct {
+	ticker         *time.Ticker
+	stopped        chan bool
+	metricsHistory *history.InternalMetricsHistory
+}
+
+func NewInternalMetricsCollector(hcInfo *history.InternalMetricsHistory) *internalMetricsCollector {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &internalMetricsCollector{
+		ticker,
+		finished,
+		hcInfo,
+	}
+}
+
+func (u *internalMetricsCollector) StartService() {
+	go func() {
+		for {
+			select {
+			case <-u.stopped:
+				return
+			case <-u.ticker.C:
+				log.Logger().Debug("Adding current status to historical partition data")
+
+				totalAppsRunning, err := m.scheduler.getTotalApplicationsRunning()
+				if err != nil {
+					log.Logger().Warn("Could not encode totalApplications metric.", zap.Error(err))
+					continue
 
 Review comment:
   Doesn't this mean that we skip collecting the containers and that we _lose_ a tick worth of data?
   In the other case if containers fail we drop the apps running info.
   If that is intentional then we need to be very clear about that but I think we should report it anyway as a 0 or -1 value etc, up to you to come up with a good concept here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401536096
 
 

 ##########
 File path: pkg/metrics/history/internal_metrics.go
 ##########
 @@ -40,35 +45,55 @@ type MetricsRecord struct {
 
 func NewInternalMetricsHistory(limit int) *InternalMetricsHistory {
 	return &InternalMetricsHistory{
-		records: make([]*MetricsRecord, 0),
+		records: make([]*MetricsRecord, limit),
 		limit:   limit,
 	}
 }
 
 func (h *InternalMetricsHistory) Store(totalApplications, totalContainers int) {
-	h.mutex.Lock()
-	defer h.mutex.Unlock()
+	h.Lock()
+	defer h.Unlock()
+
+	h.records[h.pointer] = &MetricsRecord{
+		time.Now(),
+		totalApplications,
+		totalContainers,
+	}
 
-	h.records = append(h.records,
-		&MetricsRecord{
-			time.Now(),
-			totalApplications,
-			totalContainers,
-		})
-	if len(h.records) > h.limit {
-		// remove oldest entry
-		h.records = h.records[1:]
+	h.pointer++
+	if h.pointer == h.limit {
+		h.pointer = 0
+		h.full = true
 	}
 }
 
+// the return of this function is ordered by the time of addition
 func (h *InternalMetricsHistory) GetRecords() []*MetricsRecord {
-	h.mutex.Lock()
-	defer h.mutex.Unlock()
-	return h.records
+	h.RLock()
+	defer h.RUnlock()
+
+	returnRecords := make([]*MetricsRecord, 0)
+	if h.full {
+		pointer := h.pointer
+		for i := 0; i < h.limit; i++ {
+			returnRecords = append(returnRecords, h.records[pointer])
+			pointer++
+			if pointer == h.limit {
+				pointer = 0
+			}
+		}
+	} else {
+		for _, record := range h.records {
+			if record != nil {
+				returnRecords = append(returnRecords, record)
+			}
+		}
+	}
 
 Review comment:
   Indeed, I fixed accordingly.
   
   I'd be more comfortable with filtering the nil values here, assuming we could have other caller's of this function, but right now it is ok to do it in the webapp.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402087681
 
 

 ##########
 File path: go.mod
 ##########
 @@ -13,9 +13,10 @@ require (
 	github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e // indirect
 	github.com/pkg/errors v0.9.1 // indirect
 	github.com/prometheus/client_golang v1.4.0
+	github.com/prometheus/client_model v0.2.0
 	github.com/prometheus/common v0.9.1
 	github.com/satori/go.uuid v1.2.0
-	github.com/stretchr/testify v1.5.1 // indirect
+	github.com/stretchr/testify v1.5.1
 
 Review comment:
   We have made a decision to only use gotest.tools in our testing code.
   The lint checker will even complain when you add it as an import.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r400774846
 
 

 ##########
 File path: pkg/metrics/metrics_collector.go
 ##########
 @@ -0,0 +1,89 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"time"
+
+	dto "github.com/prometheus/client_model/go"
+	"go.uber.org/zap"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/log"
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+)
+
+var tickerDefault = 1 * time.Minute
+
+type InternalMetricsCollector struct {
+	ticker         *time.Ticker
+	stopped        chan bool
+	metricsHistory *history.InternalMetricsHistory
+}
+
+func NewInternalMetricsCollector(hcInfo *history.InternalMetricsHistory) *InternalMetricsCollector {
+	finished := make(chan bool)
+	ticker := time.NewTicker(tickerDefault)
+
+	return &InternalMetricsCollector{
+		ticker,
+		finished,
+		hcInfo,
+	}
+}
+
+func (u *InternalMetricsCollector) StartService() {
+	go func() {
+		for {
+			select {
+			case <-u.stopped:
+				return
+			case <-u.ticker.C:
+				log.Logger().Debug("Adding current status to historical partition data")
+
+				totalAppsRunningMetric := &dto.Metric{}
+				totalAppsRunningMetricGauge := m.scheduler.getTotalApplicationsRunning()
+				err := (*totalAppsRunningMetricGauge).Write(totalAppsRunningMetric)
+				if err != nil {
+					log.Logger().Warn("Could not encode metric.", zap.Error(err))
+					continue
+				}
+
+				totalContainersRunningMetric := &dto.Metric{}
+				totalContainersRunningMetricCounter := m.scheduler.getAllocatedContainers()
+				err = (*totalContainersRunningMetricCounter).Write(totalContainersRunningMetric)
+				if err != nil {
+					log.Logger().Warn("Could not encode metric.", zap.Error(err))
+					continue
+				}
+
+				u.metricsHistory.Store(
+					int(*totalAppsRunningMetric.Gauge.Value),
+					int(*totalContainersRunningMetric.Counter.Value))
+			}
+		}
+	}()
+}
+
+func (u *InternalMetricsCollector) Stop() {
+	u.stopped <- true
+}
+
+func setInternalMetricsCollectorTickerForTest(newDefault time.Duration) {
+	tickerDefault = newDefault
+}
 
 Review comment:
   missing newline

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r398555717
 
 

 ##########
 File path: pkg/cache/history/history_info.go
 ##########
 @@ -0,0 +1,56 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 history
+
+import (
+	"time"
+)
+
+var LIMIT = 144 // 10 minutes in 1 day
+
+type HistoricalPartitionInfo struct {
+	records []*PartitionInfoRecord
+}
+
+type PartitionInfoRecord struct {
+	Timestamp         time.Time
+	TotalApplications int
+	TotalContainers   int
+}
+
+func (hpInfo *HistoricalPartitionInfo) Store(totalApplications, totalContainers int) {
 
 Review comment:
   Done (for `HistoricalClusterInfo`)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org


[GitHub] [incubator-yunikorn-core] adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-604518096
 
 
   Fixed the bug and tried out locally.
   
   Sample output of `localhost:9080/ws/v1/history/containers`:
   ```json
   [
       {
           "timestamp": 1585238263802408000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585238323803183000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585238383803268000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585238443804046000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585238503806253000,
           "totalContainers": "0"
       },
       {
           "timestamp": 1585238563807452000,
           "totalContainers": "0"
       }
   ]
   ```
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r401306415
 
 

 ##########
 File path: pkg/webservice/handlers.go
 ##########
 @@ -236,3 +236,39 @@ func getNodeJSON(nodeInfo *cache.NodeInfo) *dao.NodeDAOInfo {
 		Schedulable: nodeInfo.IsSchedulable(),
 	}
 }
+
+func GetApplicationHistory(w http.ResponseWriter, r *http.Request) {
+	writeHeaders(w)
+
+	var result []*dao.ApplicationHistoryDAOInfo
+	records := imHistory.GetRecords()
+	for _, record := range records {
+		element := &dao.ApplicationHistoryDAOInfo{
+			Timestamp:         record.Timestamp.UnixNano(),
+			TotalApplications: strconv.Itoa(record.TotalApplications),
 
 Review comment:
   `go fmt` or `go imports` should handle all that for us, it looks correct: tabs in front, spaces to line up 2nd part.
   To confirm you can run `~/go/bin/goimports -local "github.com/apache/incubator-yunikorn" -w .` from the root of the tree and that fixes all the issues and makes sure lint passes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
wilfred-s commented on a change in pull request #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#discussion_r402091376
 
 

 ##########
 File path: pkg/metrics/metrics_collector_test.go
 ##########
 @@ -0,0 +1,62 @@
+/*
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you 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 metrics
+
+import (
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-yunikorn-core/pkg/metrics/history"
+	"github.com/stretchr/testify/assert"
 
 Review comment:
   use gotest.tools 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-yunikorn-core] adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info

Posted by GitBox <gi...@apache.org>.
adamantal commented on issue #109: [YUNIKORN-14] Add rest API to retrieve app/container history info
URL: https://github.com/apache/incubator-yunikorn-core/pull/109#issuecomment-604426009
 
 
   Strangely the test passes on my local - I didn't touch `scheduler_recovery_test.go`, so I think it is probably unrelated.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: dev-help@yunikorn.apache.org