Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CASSSIDECAR-162: Sidecar endpoint to GET sstable's preemptive interval value #152

Open
wants to merge 6 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.io.IOException;
import java.net.UnknownHostException;
import java.security.InvalidParameterException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
Expand All @@ -29,8 +30,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.cassandra.sidecar.common.response.GetPreemptiveOpenIntervalResponse;
import org.apache.cassandra.sidecar.common.response.RingResponse;
import org.apache.cassandra.sidecar.common.response.TokenRangeReplicasResponse;
import org.apache.cassandra.sidecar.common.server.DataStorageUnit;
import org.apache.cassandra.sidecar.common.server.JmxClient;
import org.apache.cassandra.sidecar.common.server.StorageOperations;
import org.apache.cassandra.sidecar.common.server.cluster.locator.Partitioners;
Expand Down Expand Up @@ -213,4 +216,17 @@ public void outOfRangeDataCleanup(@NotNull String keyspace, @NotNull String tabl
jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME)
.forceKeyspaceCleanup(concurrency, keyspace, table);
}

@Override
public GetPreemptiveOpenIntervalResponse getSSTablePreemptiveOpenInterval(DataStorageUnit unit)
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed 'inMB' from StorageOperations function names to make them generic for supporting other units later on.

{
if (unit != DataStorageUnit.MEBIBYTES)
{
throw new InvalidParameterException("Unsupported data storage unit: " + unit);
}

int preemptiveOpenInterval = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME)
.getSSTablePreemptiveOpenIntervalInMB();
return new GetPreemptiveOpenIntervalResponse(preemptiveOpenInterval);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -159,4 +159,10 @@ public void ensureGossipIsEnabled()
LOGGER.warn("Gossip is disabled and unavailable for the operation");
throw OperationUnavailableException.GOSSIP_DISABLED;
}

@Override
public int getSSTablePreemptiveOpenIntervalInMB()
{
return delegate.getSSTablePreemptiveOpenIntervalInMB();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -159,4 +159,10 @@ public interface StorageJmxOperations
* @throws InterruptedException it does not really throw but declared in MBean
*/
int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;

/**
* Invokes C* StorageServiceMBean's JMX function getSSTablePreemptiveOpenIntervalInMB
* @return the same value returned by the JMX operation
*/
int getSSTablePreemptiveOpenIntervalInMB();
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the only place we use 'inMB', this should exactly match C* JMX function name

}
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,11 @@ public final class ApiEndpointsV1

public static final String CONNECTED_CLIENT_STATS_ROUTE = API_V1 + CASSANDRA + "/stats/connected-clients";

// Endpoint to retrieve sstable's preemptiveOpenInterval value.
// Value returned is in MB, may return negative value when disabled
private static final String SSTABLE = "/sstable";
public static final String SSTABLE_PREEMPTIVE_OPEN_INTERVAL_ROUTE = API_V1 + CASSANDRA + SSTABLE +
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know we are reusing the getSsTablePreemtiveOpenIntervalInMB JMX operation, but, from a pure API standpoint, we are omitting the MB part in the endpoint, but returning it as part of the response (SSTablePreemptiveOpenIntervalInMB). My suggestion for this would be to either:

  • We add the MB part to the endpoint (/preemptive-open-interval-in-mb)
    Or
  • We just support an optional unit filtering parameter in the url, that can be extended in the future with other unit types if needed. That way, the url would end in /preemptive-open-interval?unit=mb (remember that the unit is optional with MB as a default, so no need to pass it from the client).

That would make us have to modify the response to something like:

{
  "SSTablePreemptiveOpenInterval": 1234,
  "unit": "MB"
}

What do you think?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • We want to make endpoint generic, so underlying changes to C* shouldn't impact the endpoint usage for the end user, for example C* 6.0 returning value in different unit, while older versions continue to return the value in MB, the same endpoint should work regardless of the running c* version. Hence not including MB in the endpoint name. Whereas JMX operation name need to match exactly the JMX endpoint we are invoking. The same with passing option param like unit=MB, better we avoid supporting new unit param in the endpoint whenever C* changes for different versions.

  • In the response we need to somehow tell the end user that value returned is in in MB. I preferred the name SSTablePreemptiveOpenIntervalInMB as it matches the JMX operation currently the end users are calling. Your suggestion of separating the unit into another field sounds good, but that may complicate callers's code, as they need to parse unit as a string and convert value into those units accordingly. C* latest code changed this yaml param to for example "60MiB", let me see which one is simpler for the calling code "60MiB" or {60, "MiB"} and will get back to you

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did explore this a little, I believe the one you suggested /preemptive-open-interval?unit=mb is more easy way to implement and also is in-line with current JMX call semantics. Other alternatives unnecessarily complicates conversions from human readable format to byte size at the client.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added unit=MiB param, unit names matches with names C* is using.

"/preemptive-open-interval";
private ApiEndpointsV1()
{
throw new IllegalStateException(getClass() + " is a constants container and shall not be instantiated");
Expand Down
Original file line number Diff line number Diff line change
@@ -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 org.apache.cassandra.sidecar.common.request;

import io.netty.handler.codec.http.HttpMethod;
import org.apache.cassandra.sidecar.common.ApiEndpointsV1;
import org.apache.cassandra.sidecar.common.response.GetPreemptiveOpenIntervalResponse;

/**
* Request to get preemptive open interval value
*/
public class GetPreemptiveOpenIntervalRequest extends JsonRequest<GetPreemptiveOpenIntervalResponse>
{
/**
* Constructs a request to GET preemptive open interval value
*/
public GetPreemptiveOpenIntervalRequest()
{
super(ApiEndpointsV1.SSTABLE_PREEMPTIVE_OPEN_INTERVAL_ROUTE);
}

/**
* Constructs a request to GET preemptive open interval value with query param 'unit'
* @param unit query param, data storage units of preemptive open interval value
*/
public GetPreemptiveOpenIntervalRequest(String unit)
{
super(ApiEndpointsV1.SSTABLE_PREEMPTIVE_OPEN_INTERVAL_ROUTE + "?unit=" + unit);
}

/**
* {@inheritDoc}
*/
@Override
public HttpMethod method()
{
return HttpMethod.GET;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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 org.apache.cassandra.sidecar.common.response;

import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;

/**
* Response for GET preemptive open interval API
*/
@JsonInclude(JsonInclude.Include.NON_NULL)
public class GetPreemptiveOpenIntervalResponse
{
private final int sstablePreemptiveOpenInterval;

/**
* Constructs a response with preemptive open interval value
* @param sstablePreemptiveOpenInterval value of preemptive open interval
*/
public GetPreemptiveOpenIntervalResponse(@JsonProperty("SSTablePreemptiveOpenInterval") int sstablePreemptiveOpenInterval)
{
this.sstablePreemptiveOpenInterval = sstablePreemptiveOpenInterval;
}

@JsonProperty("SSTablePreemptiveOpenInterval")
public int sstablePreemptiveOpenInterval()
{
return sstablePreemptiveOpenInterval;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.cassandra.sidecar.common.request.ClearSnapshotRequest;
import org.apache.cassandra.sidecar.common.request.ConnectedClientStatsRequest;
import org.apache.cassandra.sidecar.common.request.CreateSnapshotRequest;
import org.apache.cassandra.sidecar.common.request.GetPreemptiveOpenIntervalRequest;
import org.apache.cassandra.sidecar.common.request.GossipInfoRequest;
import org.apache.cassandra.sidecar.common.request.ImportSSTableRequest;
import org.apache.cassandra.sidecar.common.request.ListSnapshotFilesRequest;
Expand Down Expand Up @@ -486,6 +487,28 @@ public Builder connectedClientStatsRequest()
return request(new ConnectedClientStatsRequest());
}

/**
* Sets the {@code request} to be a {@link GetPreemptiveOpenIntervalRequest} and returns a reference to this builder
* enabling method chaining
* @return a reference to this builder
*/
public Builder getPreemptiveOpenIntervalRequest()
{
return request(new GetPreemptiveOpenIntervalRequest());
}

/**
* Sets the {@code request} to be a {@link GetPreemptiveOpenIntervalRequest} with query param 'unit' and
* returns a reference to this builder enabling method chaining
* @param unit query param, data storage units of preemptive open interval value
* @return a reference to this builder
*/
public Builder getPreemptiveOpenIntervalRequest(String unit)
{
return request(new GetPreemptiveOpenIntervalRequest(unit));
}


/**
* Sets the {@code retryPolicy} to be an
* {@link org.apache.cassandra.sidecar.client.retry.ExponentialBackoffRetryPolicy} configured with
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import org.apache.cassandra.sidecar.common.request.data.RestoreJobProgressRequestParams;
import org.apache.cassandra.sidecar.common.request.data.UpdateRestoreJobRequestPayload;
import org.apache.cassandra.sidecar.common.response.ConnectedClientStatsResponse;
import org.apache.cassandra.sidecar.common.response.GetPreemptiveOpenIntervalResponse;
import org.apache.cassandra.sidecar.common.response.GossipInfoResponse;
import org.apache.cassandra.sidecar.common.response.HealthResponse;
import org.apache.cassandra.sidecar.common.response.ListSnapshotFilesResponse;
Expand Down Expand Up @@ -593,6 +594,30 @@ public CompletableFuture<ConnectedClientStatsResponse> connectedClientStats()
.build());
}

/**
* Executes GET preemptive open interval request using the default retry policy and configured selection policy
* @return a completable future with {@link GetPreemptiveOpenIntervalResponse}
*/
public CompletableFuture<GetPreemptiveOpenIntervalResponse> getPreemptiveOpenInterval()
{
return executeRequestAsync(requestBuilder()
.getPreemptiveOpenIntervalRequest()
.build());
}

/**
* Executes GET preemptive open interval request using the default retry policy and configured selection policy and
* with query param 'unit'
* @param unit query param, data storage units of preemptive open interval value
* @return a completable future with {@link GetPreemptiveOpenIntervalResponse}
*/
public CompletableFuture<GetPreemptiveOpenIntervalResponse> getPreemptiveOpenInterval(String unit)
{
return executeRequestAsync(requestBuilder()
.getPreemptiveOpenIntervalRequest(unit)
.build());
}

/**
* Returns a copy of the request builder with the default parameters configured for the client.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
import org.apache.cassandra.sidecar.common.request.data.MD5Digest;
import org.apache.cassandra.sidecar.common.request.data.XXHash32Digest;
import org.apache.cassandra.sidecar.common.response.ConnectedClientStatsResponse;
import org.apache.cassandra.sidecar.common.response.GetPreemptiveOpenIntervalResponse;
import org.apache.cassandra.sidecar.common.response.GossipInfoResponse;
import org.apache.cassandra.sidecar.common.response.HealthResponse;
import org.apache.cassandra.sidecar.common.response.ListSnapshotFilesResponse;
Expand Down Expand Up @@ -1440,6 +1441,32 @@ public void testConnectedClientStats() throws Exception
validateResponseServed(ApiEndpointsV1.CONNECTED_CLIENT_STATS_ROUTE);
}

@Test
public void testGetPreemptiveOpenInterval() throws Exception
{
String responseAsString = "{\"SSTablePreemptiveOpenInterval\":30}";

MockResponse response = new MockResponse().setResponseCode(OK.code()).setBody(responseAsString);
enqueue(response);
GetPreemptiveOpenIntervalResponse result = client.getPreemptiveOpenInterval().get();

assertThat(result).isNotNull();
assertThat(result.sstablePreemptiveOpenInterval()).isNotNull().isEqualTo(30);
}

@Test
public void testGetPreemptiveOpenIntervalWithUnit() throws Exception
{
String responseAsString = "{\"SSTablePreemptiveOpenInterval\":40}";

MockResponse response = new MockResponse().setResponseCode(OK.code()).setBody(responseAsString);
enqueue(response);
GetPreemptiveOpenIntervalResponse result = client.getPreemptiveOpenInterval("MiB").get();

assertThat(result).isNotNull();
assertThat(result.sstablePreemptiveOpenInterval()).isNotNull().isEqualTo(40);
}

private void enqueue(MockResponse response)
{
for (MockWebServer server : servers)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
package org.apache.cassandra.sidecar.common.server;

/**
* Enum for data storage units, resembles same unit names as C* is using
*/
public enum DataStorageUnit
{
BYTES("B"),
KIBIBYTES("KiB"),
MEBIBYTES("MiB"),
GIBIBYTES("GiB");

private final String unit;

DataStorageUnit(String unit)
{
this.unit = unit;
}

public String getValue()
{
return unit;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.Map;
import java.util.concurrent.ExecutionException;

import org.apache.cassandra.sidecar.common.response.GetPreemptiveOpenIntervalResponse;
import org.apache.cassandra.sidecar.common.response.RingResponse;
import org.apache.cassandra.sidecar.common.response.TokenRangeReplicasResponse;
import org.apache.cassandra.sidecar.common.server.data.Name;
Expand Down Expand Up @@ -102,4 +103,10 @@ default void outOfRangeDataCleanup(@NotNull String keyspace, @NotNull String tab
{
outOfRangeDataCleanup(keyspace, table, 1);
}

/**
* @param unit preemptive open interval value unit, only allowed value is "MiB"
* @return sstable's preemptive open interval value
*/
GetPreemptiveOpenIntervalResponse getSSTablePreemptiveOpenInterval(DataStorageUnit unit);
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.cassandra.sidecar.cluster.CassandraAdapterDelegate;
import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
import org.apache.cassandra.sidecar.common.server.MetricsOperations;
import org.apache.cassandra.sidecar.common.server.StorageOperations;
import org.apache.cassandra.sidecar.common.server.data.Name;
import org.apache.cassandra.sidecar.common.server.data.QualifiedTableName;
import org.apache.cassandra.sidecar.common.server.exceptions.JmxAuthenticationException;
Expand Down Expand Up @@ -331,4 +332,16 @@ public static String extractHostAddressWithoutPort(String address)
}
return address;
}

protected StorageOperations getStorageOperations(String host)
{
CassandraAdapterDelegate delegate = this.metadataFetcher.delegate(host);
StorageOperations storageOperations = delegate == null ? null : delegate.storageOperations();
if (storageOperations == null)
{
throw cassandraServiceUnavailable();
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we want to add some text to this error?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cassandraServiceUnavailable currently says "Cassandra service is unavailable". The same message goes back in client's response. As this is an internal error, adding further info may leak internal info to the client. We use the same cassandraServiceUnavailable almost everywhere without adding further details, must be the same reason I am assuming.

Any other kind of exception is caught in AbstractHandler's handle() and logged and updated in the response.

}

return storageOperations;
}
}
Loading