Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/integration/loki_micro_services_delete_...

324 lines
10 KiB

package integration
import (
"context"
"sort"
"strconv"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/integration/client"
"github.com/grafana/loki/integration/cluster"
"github.com/grafana/loki/pkg/storage"
)
func TestMicroServicesDeleteRequest(t *testing.T) {
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
storage.ResetBoltDBIndexClientsWithShipper()
clu := cluster.New(nil, cluster.SchemaWithBoltDBAndBoltDB)
defer func() {
assert.NoError(t, clu.Cleanup())
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
storage.ResetBoltDBIndexClientsWithShipper()
}()
// initially, run only compactor, index-gateway and distributor.
var (
tCompactor = clu.AddComponent(
"compactor",
"-target=compactor",
"-boltdb.shipper.compactor.compaction-interval=1s",
"-boltdb.shipper.compactor.retention-delete-delay=1s",
// By default, a minute is added to the delete request start time. This compensates for that.
"-boltdb.shipper.compactor.delete-request-cancel-period=-60s",
"-compactor.deletion-mode=filter-only",
"-limits.per-user-override-period=1s",
)
tDistributor = clu.AddComponent(
"distributor",
"-target=distributor",
)
)
require.NoError(t, clu.Run())
// then, run only ingester and query-scheduler.
var (
tIngester = clu.AddComponent(
"ingester",
"-target=ingester",
"-ingester.flush-on-shutdown=true",
"-ingester.wal-enabled=false",
)
tQueryScheduler = clu.AddComponent(
"query-scheduler",
"-target=query-scheduler",
"-query-scheduler.use-scheduler-ring=false",
)
)
require.NoError(t, clu.Run())
// finally, run the query-frontend and querier.
var (
tQueryFrontend = clu.AddComponent(
"query-frontend",
"-target=query-frontend",
"-frontend.scheduler-address="+tQueryScheduler.GRPCURL(),
"-frontend.default-validity=0s",
"-common.compactor-address="+tCompactor.HTTPURL(),
)
tQuerier = clu.AddComponent(
"querier",
"-target=querier",
"-querier.scheduler-address="+tQueryScheduler.GRPCURL(),
"-common.compactor-address="+tCompactor.HTTPURL(),
)
)
require.NoError(t, clu.Run())
tenantID := randStringRunes()
now := time.Now()
cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL())
cliDistributor.Now = now
cliIngester := client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL())
cliQueryFrontend.Now = now
cliCompactor := client.New(tenantID, "", tCompactor.HTTPURL())
cliCompactor.Now = now
var expectedStreams []client.StreamValues
for _, deletionType := range []string{"filter", "filter_no_match", "nothing", "partially_by_time", "whole"} {
expectedStreams = append(expectedStreams, client.StreamValues{
Stream: map[string]string{
"job": "fake",
"deletion_type": deletionType,
},
Values: [][]string{
{
strconv.FormatInt(now.Add(-48*time.Hour).UnixNano(), 10),
"lineA",
},
{
strconv.FormatInt(now.Add(-48*time.Hour).UnixNano(), 10),
"lineB",
},
{
strconv.FormatInt(now.Add(-time.Minute).UnixNano(), 10),
"lineC",
},
{
strconv.FormatInt(now.Add(-time.Minute).UnixNano(), 10),
"lineD",
},
},
})
}
expectedDeleteRequests := []client.DeleteRequest{
{
StartTime: now.Add(-48 * time.Hour).Unix(),
EndTime: now.Unix(),
Query: `{deletion_type="filter"} |= "lineB"`,
Status: "received",
},
{
StartTime: now.Add(-48 * time.Hour).Unix(),
EndTime: now.Unix(),
Query: `{deletion_type="filter_no_match"} |= "foo"`,
Status: "received",
},
{
StartTime: now.Add(-48 * time.Hour).Unix(),
EndTime: now.Add(-10 * time.Minute).Unix(),
Query: `{deletion_type="partially_by_time"}`,
Status: "received",
},
{
StartTime: now.Add(-48 * time.Hour).Unix(),
EndTime: now.Unix(),
Query: `{deletion_type="whole"}`,
Status: "received",
},
}
validateQueryResponse := func(expectedStreams []client.StreamValues, resp *client.Response) {
t.Helper()
assert.Equal(t, "streams", resp.Data.ResultType)
require.Len(t, resp.Data.Stream, len(expectedStreams))
sort.Slice(resp.Data.Stream, func(i, j int) bool {
return resp.Data.Stream[i].Stream["deletion_type"] < resp.Data.Stream[j].Stream["deletion_type"]
})
for _, stream := range resp.Data.Stream {
sort.Slice(stream.Values, func(i, j int) bool {
return stream.Values[i][1] < stream.Values[j][1]
})
}
require.Equal(t, expectedStreams, resp.Data.Stream)
}
t.Run("ingest-logs", func(t *testing.T) {
// ingest some log lines
for _, stream := range expectedStreams {
for _, val := range stream.Values {
tsNs, err := strconv.ParseInt(val[0], 10, 64)
require.NoError(t, err)
require.NoError(t, cliDistributor.PushLogLineWithTimestamp(val[1], time.Unix(0, tsNs), stream.Stream))
}
}
})
t.Run("query", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
// given default value of query_ingesters_within is 3h, older samples won't be present in the response
var es []client.StreamValues
for _, stream := range expectedStreams {
stream.Values = stream.Values[2:]
es = append(es, stream)
}
validateQueryResponse(es, resp)
})
t.Run("flush-logs-and-restart-ingester-querier", func(t *testing.T) {
// restart ingester which should flush the chunks
require.NoError(t, tIngester.Restart())
// ensure that ingester has 0 chunks in memory
cliIngester = client.New(tenantID, "", tIngester.HTTPURL())
cliIngester.Now = now
metrics, err := cliIngester.Metrics()
require.NoError(t, err)
checkMetricValue(t, "loki_ingester_chunks_flushed_total", metrics, 5)
// reset boltdb-shipper client and restart querier
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
storage.ResetBoltDBIndexClientsWithShipper()
require.NoError(t, tQuerier.Restart())
})
// Query lines
t.Run("query again to verify logs being served from storage", func(t *testing.T) {
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
validateQueryResponse(expectedStreams, resp)
})
t.Run("add-delete-requests", func(t *testing.T) {
for _, deleteRequest := range expectedDeleteRequests {
params := client.DeleteRequestParams{
Start: strconv.FormatInt(deleteRequest.StartTime, 10),
End: strconv.FormatInt(deleteRequest.EndTime, 10),
Query: deleteRequest.Query,
}
require.NoError(t, cliCompactor.AddDeleteRequest(params))
}
})
t.Run("read-delete-request", func(t *testing.T) {
deleteRequests, err := cliCompactor.GetDeleteRequests()
require.NoError(t, err)
require.ElementsMatch(t, client.DeleteRequests(expectedDeleteRequests), deleteRequests)
})
// Query lines
t.Run("verify query time filtering", func(t *testing.T) {
// reset boltdb-shipper client and restart querier
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
storage.ResetBoltDBIndexClientsWithShipper()
require.NoError(t, tQuerier.Restart())
// update expectedStreams as per the issued requests
expectedStreams[0].Values = append(expectedStreams[0].Values[:1], expectedStreams[0].Values[2:]...)
expectedStreams[3].Values = expectedStreams[3].Values[2:]
expectedStreams = expectedStreams[:4]
// query and verify that we get the resp which matches expectedStreams
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
validateQueryResponse(expectedStreams, resp)
})
// Wait until delete request is finished
t.Run("wait-until-delete-request-processed", func(t *testing.T) {
tenantLimits := tCompactor.GetTenantLimits(tenantID)
tenantLimits.DeletionMode = "filter-and-delete"
require.NoError(t, tCompactor.SetTenantLimits(tenantID, tenantLimits))
// all the delete requests should have been processed
for i := range expectedDeleteRequests {
expectedDeleteRequests[i].Status = "processed"
}
require.Eventually(t, func() bool {
deleteRequests, err := cliCompactor.GetDeleteRequests()
require.NoError(t, err)
outer:
for i := range deleteRequests {
for j := range expectedDeleteRequests {
if deleteRequests[i] == expectedDeleteRequests[j] {
continue outer
}
}
return false
}
return true
}, 20*time.Second, 1*time.Second)
// Check metrics
metrics, err := cliCompactor.Metrics()
require.NoError(t, err)
checkUserLabelAndMetricValue(t, "loki_compactor_delete_requests_processed_total", metrics, tenantID, float64(len(expectedDeleteRequests)))
// ideally this metric should be equal to 1 given that a single line matches the line filter
// but the same chunk is indexed in 3 tables
checkUserLabelAndMetricValue(t, "loki_compactor_deleted_lines", metrics, tenantID, 3)
})
// Query lines
t.Run("query-without-query-time-filtering", func(t *testing.T) {
// disable deletion for tenant to stop query time filtering of data requested for deletion
tenantLimits := tQuerier.GetTenantLimits(tenantID)
tenantLimits.DeletionMode = "disabled"
require.NoError(t, tQuerier.SetTenantLimits(tenantID, tenantLimits))
// restart querier to make it sync the index
index-shipper: add support for multiple stores (#7754) Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> **What this PR does / why we need it**: Currently loki initializes a single instance of index-shipper to [handle all the table ranges](https://github.com/grafana/loki/blob/ff7b46297345b215fbf49c2cd4c364d125b6290b/pkg/storage/factory.go#L188) (from across periods) for a given index type `boltdb-shipper, tsdb`. Since index-shipper only has the object client handle to the store defined by `shared_store_type`, it limits the index uploads to a single store. Setting `shared_store_type` to a different store at a later point in time would mean losing access to the indexes stored in the previously configured store. With this PR, we initialize a separate index-shipper & table manager for each period if `shared_store_type` is not explicity configured. This offers the flexibility to store index in multiple stores (across providers). **Note**: - usage of `shared_store_type` in this commit text refers to one of these config options depending on the index in use: `-boltdb.shipper.shared-store`, `-tsdb.shipper.shared-store` - `shared_store_type` used to default to the `object_store` from the latest `period_config` if not explicitly configured. This PR removes these defaults in favor of supporting index uploads to multiple stores. **Which issue(s) this PR fixes**: Fixes #7276 **Special notes for your reviewer**: All the instances of downloads table manager operate on the same cacheDir. But it shouldn't be a problem as the tableRanges do not overlap across periods. **Checklist** - [X] Reviewed the `CONTRIBUTING.md` guide - [ ] Documentation added - [X] Tests updated - [x] `CHANGELOG.md` updated - [x] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` --------- Signed-off-by: Ashwanth Goli <iamashwanth@gmail.com> Co-authored-by: J Stickler <julie.stickler@grafana.com>
3 years ago
storage.ResetBoltDBIndexClientsWithShipper()
require.NoError(t, tQuerier.Restart())
// ensure the deletion-mode limit is updated
require.Equal(t, "disabled", tQuerier.GetTenantLimits(tenantID).DeletionMode)
resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`)
require.NoError(t, err)
validateQueryResponse(expectedStreams, resp)
})
}
func checkUserLabelAndMetricValue(t *testing.T, metricName, metrics, tenantID string, expectedValue float64) {
t.Helper()
val, labels, err := extractMetric(metricName, metrics)
require.NoError(t, err)
require.NotNil(t, labels)
require.Len(t, labels, 1)
require.Contains(t, labels, "user")
require.Equal(t, labels["user"], tenantID)
require.Equal(t, expectedValue, val)
}
func checkMetricValue(t *testing.T, metricName, metrics string, expectedValue float64) {
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
3 years ago
t.Helper()
require.Equal(t, expectedValue, getMetricValue(t, metricName, metrics))
}
func getMetricValue(t *testing.T, metricName, metrics string) float64 {
t.Helper()
val, _, err := extractMetric(metricName, metrics)
require.NoError(t, err)
query-scheduler: fix query distribution in SSD mode (#9471) **What this PR does / why we need it**: When we run the `query-scheduler` in `ring` mode, `queriers` and `query-frontend` discover the available `query-scheduler` instances using the ring. However, we have a problem when `query-schedulers` are not running in the same process as queriers and query-frontend since [we try to get the ring client interface from the scheduler instance](https://github.com/grafana/loki/blob/abd6131bba18db7f3575241c5e6dc4eed879fbc0/pkg/loki/modules.go#L358). This causes queries not to be spread across all the available queriers when running in SSD mode because [we point querier workers to query frontend when there is no ring client and scheduler address configured](https://github.com/grafana/loki/blob/b05f4fced305800b32641ae84e3bed5f1794fa7d/pkg/querier/worker_service.go#L115). I have fixed this issue by adding a new hidden target to initialize the ring client in `reader`/`member` mode based on which service is initializing it. `reader` mode will be used by `queriers` and `query-frontend` for discovering `query-scheduler` instances from the ring. `member` mode will be used by `query-schedulers` for registering themselves in the ring. I have also made a couple of changes not directly related to the issue but it fixes some problems: * [reset metric registry for each integration test](https://github.com/grafana/loki/commit/18c4fe59078b649ad6a788a48765b101d0b97618) - Previously we were reusing the same registry for all the tests and just [ignored the attempts to register same metrics](https://github.com/grafana/loki/blob/01f0ded7fcb57e3a7b26ffc1e8e3abf04a403825/integration/cluster/cluster.go#L113). This causes the registry to have metrics registered only from the first test so any updates from subsequent tests won't reflect in the metrics. metrics was the only reliable way for me to verify that `query-schedulers` were connected to `queriers` and `query-frontend` when running in ring mode in the integration test that I added to test my changes. This should also help with other tests where earlier it was hard to reliably check the metrics. * [load config from cli as well before applying dynamic config](https://github.com/grafana/loki/commit/f9e2448fc7e718db107165cd908054c806b84337) - Previously we were applying dynamic config considering just the config from config file. This results in unexpected config changes, for example, [this config change](https://github.com/grafana/loki/blob/4148dd2c51cb827ec3889298508b95ec7731e7fd/integration/loki_micro_services_test.go#L66) was getting ignored and [dynamic config tuning was unexpectedly turning on ring mode](https://github.com/grafana/loki/blob/52cd0a39b8266564352c61ab9b845ab597008770/pkg/loki/config_wrapper.go#L94) in the config. It is better to do any config tuning based on both file and cli args configs. **Which issue(s) this PR fixes**: Fixes #9195
3 years ago
return val
}