Skip to content

Conversation

@kaijchen
Copy link
Member

What changes were proposed in this pull request?

When implementing HDDS-6518, I found KeyValueContainerData#getProtoBufMessage() does not set block count.

Although it is an optional field in ContainerData, I think we should set it to make ReadContainer command more useful.

HDDS-6518 may not get merged since another approach HDDS-6665 is taken. So I filed a separate Jira for this improvement.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-6676

How was this patch tested?

See TestReplicationService#testReadContainer in #3351.
But I don't think there is a need to add new test in this PR.

@kerneltime
Copy link
Contributor

Can we add a unit test where all attributes unless explicitly skipped are part of the serialized protobuf?

@kaijchen
Copy link
Member Author

kaijchen commented May 3, 2022

Can we add a unit test where all attributes unless explicitly skipped are part of the serialized protobuf?

I think we can check x.equals(x.fromProto(x.toProto)), do you have any better idea?

@neils-dev
Copy link
Contributor

A unit test can be added to TestKeyValueContainer (hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java). This test, TestKeyValueContainer#testKeyValueDataProtoBufMsg, can exercise the protobuf and all attributes set:
ie.

populate(numberOfKeysToWrite);
ContainerDataProto proto = containerData.getProtoBufMessage();
assertEquals(proto.getState(), keyValueContainerData.getState());
...
assertEquals(proto.getBlockCount(), numberOfKeysToWrite);

@kaijchen
Copy link
Member Author

Hi @neils-dev, thanks for the advice. I have added the test.

@adoroszlai adoroszlai merged commit ba4352b into apache:master Jun 30, 2022
@adoroszlai
Copy link
Contributor

Thanks @kaijchen for the patch, @kerneltime and @neils-dev for the review.

@kaijchen
Copy link
Member Author

kaijchen commented Jul 1, 2022

Thanks @kerneltime @neils-dev and @adoroszlai for the review.

@kaijchen kaijchen deleted the HDDS-6676 branch July 1, 2022 02:01
errose28 added a commit to errose28/ozone that referenced this pull request Jul 12, 2022
* master: (46 commits)
  HDDS-6901. Configure HDDS volume reserved as percentage of the volume space. (apache#3532)
  HDDS-6978. EC: Cleanup RECOVERING container on DN restarts (apache#3585)
  HDDS-6982. EC: Attempt to cleanup the RECOVERING container when reconstruction failed at coordinator. (apache#3583)
  HDDS-6968. Addendum: [Multi-Tenant] Fix USER_MISMATCH error even on correct user. (apache#3578)
  HDDS-6794. EC: Analyze and add putBlock even on non writing node in the case of partial single stripe. (apache#3514)
  HDDS-6900. Propagate TimeoutException for all SCM HA Ratis calls. (apache#3564)
  HDDS-6938. handle NPE when removing prefixAcl (apache#3568)
  HDDS-6960. EC: Implement the Over-replication Handler (apache#3572)
  HDDS-6979. Remove unused plexus dependency declaration (apache#3579)
  HDDS-6957. EC: ReplicationManager - priortise under replicated containers (apache#3574)
  HDDS-6723. Close Rocks objects properly in OzoneManager (apache#3400)
  HDDS-6942. Ozone Buckets/Objects created via S3 should not allow group access (apache#3553)
  HDDS-6965. Increase timeout for basic check (apache#3563)
  HDDS-6969. Add link to compose directory in smoketest README (apache#3567)
  HDDS-6970. EC: Ensure DatanodeAdminMonitor can handle EC containers during decommission (apache#3573)
  HDDS-6977. EC: Remove references to ContainerReplicaPendingOps in TestECContainerReplicaCount (apache#3575)
  HDDS-6217. Cleanup XceiverClientGrpc TODOs, and document how the client works and should be used. (apache#3012)
  HDDS-6773. Cleanup TestRDBTableStore (apache#3434) - fix checkstyle
  HDDS-6773. Cleanup TestRDBTableStore (apache#3434)
  HDDS-6676. KeyValueContainerData#getProtoBufMessage() should set block count (apache#3371)
  ...

Conflicts:
    hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants